diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java index b21068b51712b..5fcdb76c175c8 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java @@ -23,6 +23,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.index.ShardIndexingPressure; import java.io.IOException; import java.util.Collections; @@ -40,6 +41,8 @@ public class CommonStatsFlags implements Writeable, Cloneable { private String[] completionDataFields = null; private boolean includeSegmentFileSizes = false; private boolean includeUnloadedSegments = false; + private boolean includeAllShardIndexingPressureTrackers = false; + private boolean includeOnlyTopIndexingPressureMetrics = false; /** * @param flags flags to set. If no flags are supplied, default flags will be set. @@ -67,6 +70,15 @@ public CommonStatsFlags(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_7_2_0)) { includeUnloadedSegments = in.readBoolean(); } + if (in.getVersion().onOrAfter(Version.V_7_10_2)) { + includeAllShardIndexingPressureTrackers = in.readBoolean(); + includeOnlyTopIndexingPressureMetrics = in.readBoolean(); + } else if (in.getVersion().onOrAfter(Version.V_7_9_0)) { + if (ShardIndexingPressure.isShardIndexingPressureAttributeEnabled()) { + includeAllShardIndexingPressureTrackers = in.readBoolean(); + includeOnlyTopIndexingPressureMetrics = in.readBoolean(); + } + } } @Override @@ -85,6 +97,15 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_7_2_0)) { out.writeBoolean(includeUnloadedSegments); } + if (out.getVersion().onOrAfter(Version.V_7_10_2)) { + out.writeBoolean(includeAllShardIndexingPressureTrackers); + out.writeBoolean(includeOnlyTopIndexingPressureMetrics); + } else if (out.getVersion().onOrAfter(Version.V_7_9_0)) { + if (ShardIndexingPressure.isShardIndexingPressureAttributeEnabled()) { + out.writeBoolean(includeAllShardIndexingPressureTrackers); + out.writeBoolean(includeOnlyTopIndexingPressureMetrics); + } + } } /** @@ -98,6 +119,8 @@ public CommonStatsFlags all() { completionDataFields = null; includeSegmentFileSizes = false; includeUnloadedSegments = false; + includeAllShardIndexingPressureTrackers = false; + includeOnlyTopIndexingPressureMetrics = false; return this; } @@ -112,6 +135,8 @@ public CommonStatsFlags clear() { completionDataFields = null; includeSegmentFileSizes = false; includeUnloadedSegments = false; + includeAllShardIndexingPressureTrackers = false; + includeOnlyTopIndexingPressureMetrics = false; return this; } @@ -185,10 +210,28 @@ public CommonStatsFlags includeUnloadedSegments(boolean includeUnloadedSegments) return this; } + public CommonStatsFlags includeAllShardIndexingPressureTrackers(boolean includeAllShardPressureTrackers) { + this.includeAllShardIndexingPressureTrackers = includeAllShardPressureTrackers; + return this; + } + + public CommonStatsFlags includeOnlyTopIndexingPressureMetrics(boolean includeOnlyTopIndexingPressureMetrics) { + this.includeOnlyTopIndexingPressureMetrics = includeOnlyTopIndexingPressureMetrics; + return this; + } + public boolean includeUnloadedSegments() { return this.includeUnloadedSegments; } + public boolean includeAllShardIndexingPressureTrackers() { + return this.includeAllShardIndexingPressureTrackers; + } + + public boolean includeOnlyTopIndexingPressureMetrics() { + return this.includeOnlyTopIndexingPressureMetrics; + } + public boolean includeSegmentFileSizes() { return this.includeSegmentFileSizes; } diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java index f3f7363910f03..b9bb0a28b626d 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java @@ -170,7 +170,12 @@ public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteReque protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { final long indexingBytes = bulkRequest.ramBytesUsed(); final boolean isOnlySystem = isOnlySystem(bulkRequest, clusterService.state().metadata().getIndicesLookup(), systemIndices); - final Releasable releasable = indexingPressure.markCoordinatingOperationStarted(indexingBytes, isOnlySystem); + final Releasable releasable; + if (indexingPressure.isShardIndexingPressureEnabled() == false) { + releasable = indexingPressure.markCoordinatingOperationStarted(indexingBytes, isOnlySystem); + } else { + releasable = () -> {}; + } final ActionListener releasingListener = ActionListener.runBefore(listener, releasable::close); final String executorName = isOnlySystem ? Names.SYSTEM_WRITE : Names.WRITE; try { @@ -548,7 +553,17 @@ protected void doRun() { if (task != null) { bulkShardRequest.setParentTask(nodeId, task.getId()); } - shardBulkAction.execute(bulkShardRequest, new ActionListener() { + // Add the shard level accounting for coordinating and supply the listener + final Releasable releasable; + if (indexingPressure.getShardIndexingPressure().isShardIndexingPressureEnabled()) { + final boolean isOnlySystem = isOnlySystem(bulkRequest, clusterService.state().metadata().getIndicesLookup(), + systemIndices); + releasable = indexingPressure.getShardIndexingPressure() + .markCoordinatingOperationStarted(shardId, bulkShardRequest.ramBytesUsed(), isOnlySystem); + } else { + releasable = () -> {}; + } + shardBulkAction.execute(bulkShardRequest, ActionListener.runBefore(new ActionListener() { @Override public void onResponse(BulkShardResponse bulkShardResponse) { for (BulkItemResponse bulkItemResponse : bulkShardResponse.getResponses()) { @@ -581,7 +596,7 @@ private void finishHim() { listener.onResponse(new BulkResponse(responses.toArray(new BulkItemResponse[responses.length()]), buildTookInMillis(startTimeNanos))); } - }); + }, releasable::close)); } bulkRequest = null; // allow memory for bulk request items to be reclaimed before all items have been completed } diff --git a/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java index 44a2f76ef2697..3cd760cde9d08 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java @@ -288,23 +288,34 @@ private void handleOperationRequest(final Request request, final TransportChanne Releasable releasable = checkOperationLimits(request); ActionListener listener = ActionListener.runBefore(new ChannelActionListener<>(channel, actionName, request), releasable::close); - runReroutePhase(task, request, listener, false); + // Add the shard level accounting for primary and chain the listener + Releasable shardReleasable = checkShardOperationLimits(request); + ActionListener finalListener = ActionListener.runBefore(listener, shardReleasable::close); + runReroutePhase(task, request, finalListener, false); } protected Releasable checkOperationLimits(final Request request) { return () -> {}; } + protected Releasable checkShardOperationLimits(final Request request) { + return () -> {}; + } + protected void handlePrimaryRequest(final ConcreteShardRequest request, final TransportChannel channel, final Task task) { Releasable releasable = checkPrimaryLimits(request.getRequest(), request.sentFromLocalReroute(), request.localRerouteInitiatedByNodeClient()); ActionListener listener = ActionListener.runBefore(new ChannelActionListener<>(channel, transportPrimaryAction, request), releasable::close); + // Add the shard level accounting for transport primary and chain the listener + Releasable shardReleasable = checkShardPrimaryLimits(request.getRequest(), request.sentFromLocalReroute(), + request.localRerouteInitiatedByNodeClient()); + ActionListener finalListener = ActionListener.runBefore(listener, shardReleasable::close); try { - new AsyncPrimaryAction(request, listener, (ReplicationTask) task).run(); + new AsyncPrimaryAction(request, finalListener, (ReplicationTask) task).run(); } catch (RuntimeException e) { - listener.onFailure(e); + finalListener.onFailure(e); } } @@ -312,6 +323,11 @@ protected Releasable checkPrimaryLimits(final Request request, boolean rerouteWa return () -> {}; } + protected Releasable checkShardPrimaryLimits(final Request request, boolean rerouteWasLocal, + boolean localRerouteInitiatedByNodeClient) { + return () -> {}; + } + class AsyncPrimaryAction extends AbstractRunnable { private final ActionListener onCompletionListener; private final ReplicationTask replicationTask; @@ -524,11 +540,13 @@ protected void handleReplicaRequest(final ConcreteReplicaRequest Releasable releasable = checkReplicaLimits(replicaRequest.getRequest()); ActionListener listener = ActionListener.runBefore(new ChannelActionListener<>(channel, transportReplicaAction, replicaRequest), releasable::close); - + // Add the shard level accounting for replica and chain the listener + Releasable shardReleasable = checkShardReplicaLimits(replicaRequest.getRequest()); + ActionListener finalListener = ActionListener.runBefore(listener, shardReleasable::close); try { - new AsyncReplicaAction(replicaRequest, listener, (ReplicationTask) task).run(); + new AsyncReplicaAction(replicaRequest, finalListener, (ReplicationTask) task).run(); } catch (RuntimeException e) { - listener.onFailure(e); + finalListener.onFailure(e); } } @@ -536,6 +554,10 @@ protected Releasable checkReplicaLimits(final ReplicaRequest request) { return () -> {}; } + protected Releasable checkShardReplicaLimits(final ReplicaRequest request) { + return () -> {}; + } + public static class RetryOnReplicaException extends OpenSearchException { public RetryOnReplicaException(ShardId shardId, String msg) { diff --git a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java index d4f2306b81308..39ba302e71d14 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java @@ -37,6 +37,7 @@ import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.Settings; import org.opensearch.index.IndexingPressure; +import org.opensearch.index.ShardIndexingPressure; import org.opensearch.index.engine.Engine; import org.opensearch.index.mapper.MapperParsingException; import org.opensearch.index.shard.IndexShard; @@ -64,6 +65,7 @@ public abstract class TransportWriteAction< > extends TransportReplicationAction { protected final IndexingPressure indexingPressure; + private final ShardIndexingPressure shardIndexingPressure; protected final SystemIndices systemIndices; private final Function executorFunction; @@ -79,6 +81,7 @@ protected TransportWriteAction(Settings settings, String actionName, TransportSe request, replicaRequest, ThreadPool.Names.SAME, true, forceExecutionOnPrimary); this.executorFunction = executorFunction; this.indexingPressure = indexingPressure; + this.shardIndexingPressure = indexingPressure.getShardIndexingPressure(); this.systemIndices = systemIndices; } @@ -88,7 +91,20 @@ protected String executor(IndexShard shard) { @Override protected Releasable checkOperationLimits(Request request) { - return indexingPressure.markPrimaryOperationStarted(primaryOperationSize(request), force(request)); + if (indexingPressure.isShardIndexingPressureEnabled() == false) { + return indexingPressure.markPrimaryOperationStarted(primaryOperationSize(request), force(request)); + } else { + return () -> {}; + } + } + + @Override + protected Releasable checkShardOperationLimits(Request request) { + if (shardIndexingPressure.isShardIndexingPressureEnabled()) { + return shardIndexingPressure.markPrimaryOperationStarted(request.shardId, primaryOperationSize(request), force(request)); + } else { + return () -> {}; + } } protected boolean force(ReplicatedWriteRequest request) { @@ -102,19 +118,46 @@ protected boolean isSystemShard(ShardId shardId) { @Override protected Releasable checkPrimaryLimits(Request request, boolean rerouteWasLocal, boolean localRerouteInitiatedByNodeClient) { - if (rerouteWasLocal) { - // If this primary request was received from a local reroute initiated by the node client, we - // must mark a new primary operation local to the coordinating node. - if (localRerouteInitiatedByNodeClient) { - return indexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(primaryOperationSize(request)); + if (indexingPressure.isShardIndexingPressureEnabled() == false) { + if (rerouteWasLocal) { + // If this primary request was received from a local reroute initiated by the node client, we + // must mark a new primary operation local to the coordinating node. + if (localRerouteInitiatedByNodeClient) { + return indexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(primaryOperationSize(request)); + } else { + return () -> {}; + } } else { - return () -> {}; + // If this primary request was received directly from the network, we must mark a new primary + // operation. This happens if the write action skips the reroute step (ex: rsync) or during + // primary delegation, after the primary relocation hand-off. + return indexingPressure.markPrimaryOperationStarted(primaryOperationSize(request), force(request)); } } else { - // If this primary request was received directly from the network, we must mark a new primary - // operation. This happens if the write action skips the reroute step (ex: rsync) or during - // primary delegation, after the primary relocation hand-off. - return indexingPressure.markPrimaryOperationStarted(primaryOperationSize(request), force(request)); + return () -> {}; + } + } + + @Override + protected Releasable checkShardPrimaryLimits(Request request, boolean rerouteWasLocal, boolean localRerouteInitiatedByNodeClient) { + if (shardIndexingPressure.isShardIndexingPressureEnabled()) { + if (rerouteWasLocal) { + // If this primary request was received from a local reroute initiated by the node client, we + // must mark a new primary operation local to the coordinating node. + if (localRerouteInitiatedByNodeClient) { + return shardIndexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(request.shardId, + primaryOperationSize(request)); + } else { + return () -> {}; + } + } else { + // If this primary request was received directly from the network, we must mark a new primary + // operation. This happens if the write action skips the reroute step (ex: rsync) or during + // primary delegation, after the primary relocation hand-off. + return shardIndexingPressure.markPrimaryOperationStarted(request.shardId, primaryOperationSize(request), force(request)); + } + } else { + return () -> {}; } } @@ -124,7 +167,20 @@ protected long primaryOperationSize(Request request) { @Override protected Releasable checkReplicaLimits(ReplicaRequest request) { - return indexingPressure.markReplicaOperationStarted(replicaOperationSize(request), force(request)); + if (indexingPressure.isShardIndexingPressureEnabled() == false) { + return indexingPressure.markReplicaOperationStarted(replicaOperationSize(request), force(request)); + } else { + return () -> {}; + } + } + + @Override + protected Releasable checkShardReplicaLimits(ReplicaRequest request) { + if (shardIndexingPressure.isShardIndexingPressureEnabled()) { + return shardIndexingPressure.markReplicaOperationStarted(request.shardId, replicaOperationSize(request), force(request)); + } else { + return () -> {}; + } } protected long replicaOperationSize(ReplicaRequest request) { diff --git a/server/src/main/java/org/opensearch/cluster/service/ClusterService.java b/server/src/main/java/org/opensearch/cluster/service/ClusterService.java index 5c7b8ca72416e..1d8321f64252e 100644 --- a/server/src/main/java/org/opensearch/cluster/service/ClusterService.java +++ b/server/src/main/java/org/opensearch/cluster/service/ClusterService.java @@ -34,6 +34,7 @@ import org.opensearch.common.component.AbstractLifecycleComponent; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; +import org.opensearch.index.IndexingPressure; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; import org.opensearch.node.Node; @@ -65,6 +66,8 @@ public class ClusterService extends AbstractLifecycleComponent { private RerouteService rerouteService; + private IndexingPressure indexingPressure; + public ClusterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) { this(settings, clusterSettings, new MasterService(settings, clusterSettings, threadPool), new ClusterApplierService(Node.NODE_NAME_SETTING.get(settings), settings, clusterSettings, threadPool)); @@ -190,6 +193,22 @@ public MasterService getMasterService() { return masterService; } + /** + * Getter and Setter for Indexing Pressure, This method is added specifically for getting IndexingPressure + * instance in ODFE PA plugin via ClusterService. Indexing Pressure instances can be accessible only via + * Node and NodeService class but none of them are present in the createComponents signature of ES OSS Plugin + * interface. + * {@link org.opensearch.plugins.Plugin#createComponents} + */ + + public void setIndexingPressure(IndexingPressure indexingPressure) { + this.indexingPressure = indexingPressure; + } + + public IndexingPressure getIndexingPressure() { + return indexingPressure; + } + public ClusterApplierService getClusterApplierService() { return clusterApplierService; } 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 a0b0c068e8a71..3918630018591 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -88,6 +88,9 @@ import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; import org.opensearch.index.IndexingPressure; +import org.opensearch.index.ShardIndexingPressureMemoryManager; +import org.opensearch.index.ShardIndexingPressureSettings; +import org.opensearch.index.ShardIndexingPressureStore; import org.opensearch.indices.IndexingMemoryController; import org.opensearch.indices.IndicesQueryCache; import org.opensearch.indices.IndicesRequestCache; @@ -253,6 +256,18 @@ public void apply(Settings value, Settings current, Settings previous) { DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING, SameShardAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING, ShardStateAction.FOLLOW_UP_REROUTE_PRIORITY_SETTING, + ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED, + ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED, + ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW, + ShardIndexingPressureSettings.SHARD_MIN_LIMIT, + ShardIndexingPressureStore.MAX_CACHE_STORE_SIZE, + ShardIndexingPressureMemoryManager.LOWER_OPERATING_FACTOR, + ShardIndexingPressureMemoryManager.OPTIMAL_OPERATING_FACTOR, + ShardIndexingPressureMemoryManager.UPPER_OPERATING_FACTOR, + ShardIndexingPressureMemoryManager.NODE_SOFT_LIMIT, + ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS, + ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT, + ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS, InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING, InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING, InternalSnapshotsInfoService.INTERNAL_SNAPSHOT_INFO_MAX_CONCURRENT_FETCHES_SETTING, diff --git a/server/src/main/java/org/opensearch/index/IndexingPressure.java b/server/src/main/java/org/opensearch/index/IndexingPressure.java index b8ded971628ea..6015fdba5bbb5 100644 --- a/server/src/main/java/org/opensearch/index/IndexingPressure.java +++ b/server/src/main/java/org/opensearch/index/IndexingPressure.java @@ -21,6 +21,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; @@ -36,6 +37,8 @@ public class IndexingPressure { public static final Setting MAX_INDEXING_BYTES = Setting.memorySizeSetting("indexing_pressure.memory.limit", "10%", Setting.Property.NodeScope); + private final ShardIndexingPressure shardIndexingPressure; + private static final Logger logger = LogManager.getLogger(IndexingPressure.class); private final AtomicLong currentCombinedCoordinatingAndPrimaryBytes = new AtomicLong(0); @@ -55,9 +58,11 @@ public class IndexingPressure { private final long primaryAndCoordinatingLimits; private final long replicaLimits; - public IndexingPressure(Settings settings) { + public IndexingPressure(Settings settings, ClusterService clusterService) { this.primaryAndCoordinatingLimits = MAX_INDEXING_BYTES.get(settings).getBytes(); this.replicaLimits = (long) (this.primaryAndCoordinatingLimits * 1.5); + + shardIndexingPressure = new ShardIndexingPressure(this, clusterService, settings); } @@ -160,6 +165,66 @@ public long getCurrentReplicaBytes() { return currentReplicaBytes.get(); } + public long addAndGetCurrentCombinedCoordinatingAndPrimaryBytes(long bytes) { + return currentCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes); + } + + public long addAndGetCurrentCoordinatingBytes(long bytes) { + return currentCoordinatingBytes.addAndGet(bytes); + } + + public long addAndGetCurrentPrimaryBytes(long bytes) { + return currentPrimaryBytes.addAndGet(bytes); + } + + public long addAndGetCurrentReplicaBytes(long bytes) { + return currentReplicaBytes.addAndGet(bytes); + } + + public long addAndGetTotalCombinedCoordinatingAndPrimaryBytes(long bytes) { + return totalCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes); + } + + public long addAndGetTotalCoordinatingBytes(long bytes) { + return totalCoordinatingBytes.addAndGet(bytes); + } + + public long addAndGetTotalPrimaryBytes(long bytes) { + return totalPrimaryBytes.addAndGet(bytes); + } + + public long addAndGetTotalReplicaBytes(long bytes) { + return totalReplicaBytes.addAndGet(bytes); + } + + public long getAndIncrementCoordinatingRejections() { + return coordinatingRejections.getAndIncrement(); + } + + public long getAndIncrementPrimaryRejections() { + return primaryRejections.getAndIncrement(); + } + + public long getAndIncrementReplicaRejections() { + return replicaRejections.getAndIncrement(); + } + + public long getPrimaryAndCoordinatingLimits() { + return this.primaryAndCoordinatingLimits; + } + + public long getReplicaLimits() { + return this.replicaLimits; + } + + public ShardIndexingPressure getShardIndexingPressure() { + return shardIndexingPressure; + } + + public boolean isShardIndexingPressureEnabled() { + return shardIndexingPressure.isShardIndexingPressureEnabled(); + } + public IndexingPressureStats stats() { return new IndexingPressureStats(totalCombinedCoordinatingAndPrimaryBytes.get(), totalCoordinatingBytes.get(), totalPrimaryBytes.get(), totalReplicaBytes.get(), currentCombinedCoordinatingAndPrimaryBytes.get(), diff --git a/server/src/main/java/org/opensearch/index/ShardIndexingPressure.java b/server/src/main/java/org/opensearch/index/ShardIndexingPressure.java new file mode 100644 index 0000000000000..642699e35d7c2 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/ShardIndexingPressure.java @@ -0,0 +1,436 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.admin.indices.stats.CommonStatsFlags; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.stats.ShardIndexingPressureStats; +import org.opensearch.index.stats.IndexingPressurePerShardStats; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** + * Shard Indexing Pressure is the uber level class similar to IndexingPressure. + * The methods of this class will be invoked from Transport Action to start the memory accounting and as a response + * it provides Releasable which will remove those memory accounting values or perform necessary actions once the request + * completes. + * + * This class will be responsible for + * 1. Memory Accounting at shard level. + * 2. Memory Accounting at Node level. The tracking happens in the same variables defined in IndexingPressure to support + * consistency even after feature toggle. + * 3. Instantiating new tracker objects for new shards and moving the shard tracker object to cold store from hot when + * the respective criteria meet via {@link ShardIndexingPressureStore} + * 4. Calling methods of {@link ShardIndexingPressureMemoryManager} to evaluate if a request can be process successfully + * and can increase the memory limits for a shard under certain scenarios + */ +public class ShardIndexingPressure { + + public static final String SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY = "shard_indexing_pressure_enabled"; + private final Logger logger = LogManager.getLogger(getClass()); + + private final IndexingPressure indexingPressure; + private static ClusterService clusterService; + private final ShardIndexingPressureSettings shardIndexingPressureSettings; + private final ShardIndexingPressureMemoryManager memoryManager; + private final ShardIndexingPressureStore shardIndexingPressureStore; + + ShardIndexingPressure(IndexingPressure indexingPressure, ClusterService clusterService, Settings settings) { + shardIndexingPressureSettings = new ShardIndexingPressureSettings(clusterService.getClusterSettings(), settings, + indexingPressure.getPrimaryAndCoordinatingLimits()); + ShardIndexingPressure.clusterService = clusterService; + this.indexingPressure = indexingPressure; + ClusterSettings clusterSettings = clusterService.getClusterSettings(); + + this.memoryManager = new ShardIndexingPressureMemoryManager(shardIndexingPressureSettings, clusterSettings, settings); + this.shardIndexingPressureStore = new ShardIndexingPressureStore(shardIndexingPressureSettings, clusterSettings, settings); + } + + public Releasable markCoordinatingOperationStarted(ShardId shardId, long bytes, boolean forceExecution) { + if(0 == bytes) { return () -> {}; } + + long requestStartTime = System.currentTimeMillis(); + ShardIndexingPressureTracker tracker = getShardIndexingPressureTracker(shardId); + long nodeCombinedBytes = indexingPressure.addAndGetCurrentCombinedCoordinatingAndPrimaryBytes(bytes); + long nodeReplicaBytes = indexingPressure.getCurrentReplicaBytes(); + long nodeTotalBytes = nodeCombinedBytes + nodeReplicaBytes; + long shardCombinedBytes = tracker.currentCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes); + + boolean shardLevelLimitBreached = false; + if (!forceExecution) { + boolean nodeLevelLimitBreached = memoryManager.isCoordinatingNodeLimitBreached(tracker, nodeTotalBytes); + if (!nodeLevelLimitBreached) { + shardLevelLimitBreached = memoryManager.isCoordinatingShardLimitBreached(tracker, requestStartTime, + shardIndexingPressureStore.getShardIndexingPressureHotStore(), nodeTotalBytes); + } + boolean shouldRejectRequest = nodeLevelLimitBreached || + (shardLevelLimitBreached && shardIndexingPressureSettings.isShardIndexingPressureEnforced()); + + if (shouldRejectRequest) { + long nodeBytesWithoutOperation = nodeCombinedBytes - bytes; + long nodeTotalBytesWithoutOperation = nodeTotalBytes - bytes; + long shardBytesWithoutOperation = shardCombinedBytes - bytes; + + indexingPressure.addAndGetCurrentCombinedCoordinatingAndPrimaryBytes(-bytes); + indexingPressure.getAndIncrementCoordinatingRejections(); + tracker.currentCombinedCoordinatingAndPrimaryBytes.getAndAdd(-bytes); + tracker.coordinatingRejections.getAndIncrement(); + + throw new OpenSearchRejectedExecutionException("rejected execution of coordinating operation [" + + "shard_detail=[" + shardId.getIndexName() + "][" + shardId.id() + "][C], " + + "shard_coordinating_and_primary_bytes=" + shardBytesWithoutOperation + ", " + + "shard_operation_bytes=" + bytes + ", " + + "shard_max_coordinating_and_primary_bytes=" + tracker.primaryAndCoordinatingLimits + "] OR [" + + "node_coordinating_and_primary_bytes=" + nodeBytesWithoutOperation + ", " + + "node_replica_bytes=" + nodeReplicaBytes + ", " + + "node_all_bytes=" + nodeTotalBytesWithoutOperation + ", " + + "node_operation_bytes=" + bytes + ", " + + "node_max_coordinating_and_primary_bytes=" + this.indexingPressure.getPrimaryAndCoordinatingLimits() + "]", false); + } + } + indexingPressure.addAndGetCurrentCoordinatingBytes(bytes); + indexingPressure.addAndGetTotalCombinedCoordinatingAndPrimaryBytes(bytes); + indexingPressure.addAndGetTotalCoordinatingBytes(bytes); + tracker.currentCoordinatingBytes.getAndAdd(bytes); + tracker.coordinatingCount.incrementAndGet(); + tracker.totalOutstandingCoordinatingRequests.incrementAndGet(); + + // In shadow mode if request was intended to rejected; it should only contribute to accounting limits and + // should not influence dynamic parameters such as throughput + if (shardLevelLimitBreached) { + return () -> { + indexingPressure.addAndGetCurrentCombinedCoordinatingAndPrimaryBytes(-bytes); + indexingPressure.addAndGetCurrentCoordinatingBytes(-bytes); + tracker.currentCombinedCoordinatingAndPrimaryBytes.addAndGet(-bytes); + tracker.currentCoordinatingBytes.addAndGet(-bytes); + tracker.totalCombinedCoordinatingAndPrimaryBytes.getAndAdd(bytes); + tracker.totalCoordinatingBytes.getAndAdd(bytes); + + memoryManager.decreaseShardPrimaryAndCoordinatingLimits(tracker); + shardIndexingPressureStore.tryIndexingPressureTrackerCleanup(tracker); + }; + } + + return () -> { + long requestEndTime = System.currentTimeMillis(); + long requestLatency = requestEndTime - requestStartTime; + + indexingPressure.addAndGetCurrentCombinedCoordinatingAndPrimaryBytes(-bytes); + indexingPressure.addAndGetCurrentCoordinatingBytes(-bytes); + tracker.currentCombinedCoordinatingAndPrimaryBytes.addAndGet(-bytes); + tracker.currentCoordinatingBytes.addAndGet(-bytes); + tracker.coordinatingTimeInMillis.addAndGet(requestLatency); + tracker.totalCombinedCoordinatingAndPrimaryBytes.getAndAdd(bytes); + tracker.totalCoordinatingBytes.getAndAdd(bytes); + tracker.lastSuccessfulCoordinatingRequestTimestamp.set(requestEndTime); + tracker.totalOutstandingCoordinatingRequests.set(0); + + if(requestLatency > 0) { + double requestThroughput = (double) bytes / requestLatency; + tracker.coordinatingThroughputMovingQueue.offer(requestThroughput); + if (tracker.coordinatingThroughputMovingQueue.size() > shardIndexingPressureSettings.getRequestSizeWindow()) { + double front = tracker.coordinatingThroughputMovingQueue.poll(); + double movingAverage = calculateMovingAverage(tracker.coordinatingThroughputMovingAverage.get(), + front, requestThroughput, shardIndexingPressureSettings.getRequestSizeWindow()); + tracker.coordinatingThroughputMovingAverage.set(Double.doubleToLongBits(movingAverage)); + } else { + double movingAverage = (double) tracker.totalCoordinatingBytes.get() / tracker.coordinatingTimeInMillis.get(); + tracker.coordinatingThroughputMovingAverage.set(Double.doubleToLongBits(movingAverage)); + } + } + memoryManager.decreaseShardPrimaryAndCoordinatingLimits(tracker); + shardIndexingPressureStore.tryIndexingPressureTrackerCleanup(tracker); + }; + } + + public Releasable markPrimaryOperationLocalToCoordinatingNodeStarted(ShardId shardId, long bytes) { + if(bytes == 0) { return () -> {}; } + + ShardIndexingPressureTracker tracker = getShardIndexingPressureTracker(shardId); + + indexingPressure.addAndGetCurrentPrimaryBytes(bytes); + indexingPressure.addAndGetTotalPrimaryBytes(bytes); + tracker.currentPrimaryBytes.getAndAdd(bytes); + tracker.totalPrimaryBytes.getAndAdd(bytes); + + return () -> { + indexingPressure.addAndGetCurrentPrimaryBytes(-bytes); + tracker.currentPrimaryBytes.addAndGet(-bytes); + }; + } + + public Releasable markPrimaryOperationStarted(ShardId shardId, long bytes, boolean forceExecution) { + if(0 == bytes) { return () -> {}; } + + long requestStartTime = System.currentTimeMillis(); + ShardIndexingPressureTracker tracker = getShardIndexingPressureTracker(shardId); + long nodeCombinedBytes = indexingPressure.addAndGetCurrentCombinedCoordinatingAndPrimaryBytes(bytes); + long nodeReplicaBytes = indexingPressure.getCurrentReplicaBytes(); + long nodeTotalBytes = nodeCombinedBytes + nodeReplicaBytes; + long shardCombinedBytes = tracker.currentCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes); + + boolean shardLevelLimitBreached = false; + if (!forceExecution) { + boolean nodeLevelLimitBreached = memoryManager.isPrimaryNodeLimitBreached(tracker, nodeTotalBytes); + if (!nodeLevelLimitBreached) { + shardLevelLimitBreached = memoryManager.isPrimaryShardLimitBreached(tracker, requestStartTime, + shardIndexingPressureStore.getShardIndexingPressureHotStore(), nodeTotalBytes); + } + boolean shouldRejectRequest = nodeLevelLimitBreached || + (shardLevelLimitBreached && shardIndexingPressureSettings.isShardIndexingPressureEnforced()); + + if (shouldRejectRequest) { + long nodeBytesWithoutOperation = nodeCombinedBytes - bytes; + long nodeTotalBytesWithoutOperation = nodeTotalBytes - bytes; + long shardBytesWithoutOperation = shardCombinedBytes - bytes; + + indexingPressure.addAndGetCurrentCombinedCoordinatingAndPrimaryBytes(-bytes); + indexingPressure.getAndIncrementPrimaryRejections(); + tracker.currentCombinedCoordinatingAndPrimaryBytes.getAndAdd(-bytes); + tracker.primaryRejections.getAndIncrement(); + + throw new OpenSearchRejectedExecutionException("rejected execution of primary operation [" + + "shard_detail=[" + shardId.getIndexName() + "][" + shardId.id() + "][P], " + + "shard_coordinating_and_primary_bytes=" + shardBytesWithoutOperation + ", " + + "shard_operation_bytes=" + bytes + ", " + + "shard_max_coordinating_and_primary_bytes=" + tracker.primaryAndCoordinatingLimits + "] OR [" + + "node_coordinating_and_primary_bytes=" + nodeBytesWithoutOperation + ", " + + "node_replica_bytes=" + nodeReplicaBytes + ", " + + "node_all_bytes=" + nodeTotalBytesWithoutOperation + ", " + + "node_operation_bytes=" + bytes + ", " + + "node_max_coordinating_and_primary_bytes=" + this.indexingPressure.getPrimaryAndCoordinatingLimits() + "]", false); + } + } + indexingPressure.addAndGetCurrentPrimaryBytes(bytes); + indexingPressure.addAndGetTotalCombinedCoordinatingAndPrimaryBytes(bytes); + indexingPressure.addAndGetTotalPrimaryBytes(bytes); + tracker.currentPrimaryBytes.getAndAdd(bytes); + tracker.primaryCount.incrementAndGet(); + tracker.totalOutstandingPrimaryRequests.incrementAndGet(); + + // In shadow mode if request was intended to rejected; it should only contribute to accounting limits and + // should not influence dynamic parameters such as throughput + if (shardLevelLimitBreached) { + return () -> { + indexingPressure.addAndGetCurrentCombinedCoordinatingAndPrimaryBytes(-bytes); + indexingPressure.addAndGetCurrentPrimaryBytes(-bytes); + tracker.currentCombinedCoordinatingAndPrimaryBytes.addAndGet(-bytes); + tracker.currentPrimaryBytes.addAndGet(-bytes); + tracker.totalCombinedCoordinatingAndPrimaryBytes.getAndAdd(bytes); + tracker.totalPrimaryBytes.getAndAdd(bytes); + + memoryManager.decreaseShardPrimaryAndCoordinatingLimits(tracker); + shardIndexingPressureStore.tryIndexingPressureTrackerCleanup(tracker); + }; + } + + return () -> { + long requestEndTime = System.currentTimeMillis(); + long requestLatency = requestEndTime - requestStartTime; + + indexingPressure.addAndGetCurrentCombinedCoordinatingAndPrimaryBytes(-bytes); + indexingPressure.addAndGetCurrentPrimaryBytes(-bytes); + tracker.currentCombinedCoordinatingAndPrimaryBytes.addAndGet(-bytes); + tracker.currentPrimaryBytes.addAndGet(-bytes); + tracker.primaryTimeInMillis.addAndGet(requestLatency); + tracker.totalCombinedCoordinatingAndPrimaryBytes.getAndAdd(bytes); + tracker.totalPrimaryBytes.getAndAdd(bytes); + tracker.lastSuccessfulPrimaryRequestTimestamp.set(requestEndTime); + tracker.totalOutstandingPrimaryRequests.set(0); + + if(requestLatency > 0) { + double requestThroughput = (double)bytes / requestLatency; + tracker.primaryThroughputMovingQueue.offer(requestThroughput); + if(tracker.primaryThroughputMovingQueue.size() > shardIndexingPressureSettings.getRequestSizeWindow()) { + double front = tracker.primaryThroughputMovingQueue.poll(); + double movingAverage = calculateMovingAverage(tracker.primaryThroughputMovingAverage.get(), front, + requestThroughput, shardIndexingPressureSettings.getRequestSizeWindow()); + tracker.primaryThroughputMovingAverage.set(Double.doubleToLongBits(movingAverage)); + } else { + double movingAverage = (double) tracker.totalPrimaryBytes.get() / tracker.primaryTimeInMillis.get(); + tracker.primaryThroughputMovingAverage.set(Double.doubleToLongBits(movingAverage)); + } + } + memoryManager.decreaseShardPrimaryAndCoordinatingLimits(tracker); + shardIndexingPressureStore.tryIndexingPressureTrackerCleanup(tracker); + }; + } + + public Releasable markReplicaOperationStarted(ShardId shardId, long bytes, boolean forceExecution) { + if(0 == bytes) { return () -> {}; } + + long requestStartTime = System.currentTimeMillis(); + ShardIndexingPressureTracker tracker = getShardIndexingPressureTracker(shardId); + long nodeReplicaBytes = indexingPressure.addAndGetCurrentReplicaBytes(bytes); + long shardReplicaBytes = tracker.currentReplicaBytes.addAndGet(bytes); + + boolean shardLevelLimitBreached = false; + if (!forceExecution) { + boolean nodeLevelLimitBreached = memoryManager.isReplicaNodeLimitBreached(tracker, nodeReplicaBytes); + if (!nodeLevelLimitBreached) { + shardLevelLimitBreached = memoryManager.isReplicaShardLimitBreached(tracker, requestStartTime, + shardIndexingPressureStore.getShardIndexingPressureHotStore(), nodeReplicaBytes); + } + boolean shouldRejectRequest = nodeLevelLimitBreached || + (shardLevelLimitBreached && shardIndexingPressureSettings.isShardIndexingPressureEnforced()); + + if (shouldRejectRequest) { + long nodeReplicaBytesWithoutOperation = nodeReplicaBytes - bytes; + long shardReplicaBytesWithoutOperation = shardReplicaBytes - bytes; + + indexingPressure.addAndGetCurrentReplicaBytes(-bytes); + indexingPressure.getAndIncrementReplicaRejections(); + tracker.currentReplicaBytes.getAndAdd(-bytes); + tracker.replicaRejections.getAndIncrement(); + + throw new OpenSearchRejectedExecutionException("rejected execution of replica operation [" + + "shard_detail=[" + shardId.getIndexName() + "][" + shardId.id() + "][R], " + + "shard_replica_bytes=" + shardReplicaBytesWithoutOperation + ", " + + "operation_bytes=" + bytes + ", " + + "max_coordinating_and_primary_bytes=" + tracker.replicaLimits + "] OR [" + + "replica_bytes=" + nodeReplicaBytesWithoutOperation + ", " + + "operation_bytes=" + bytes + ", " + + "max_coordinating_and_primary_bytes=" + this.indexingPressure.getReplicaLimits() + "]", false); + } + } + indexingPressure.addAndGetTotalReplicaBytes(bytes); + tracker.replicaCount.incrementAndGet(); + tracker.totalOutstandingReplicaRequests.incrementAndGet(); + + // In shadow-mode if request was intended to rejected; it should only contribute to accounting limits and + // should not influence dynamic parameters such as throughput + if (shardLevelLimitBreached) { + return () -> { + indexingPressure.addAndGetCurrentReplicaBytes(-bytes); + tracker.currentReplicaBytes.addAndGet(-bytes); + tracker.totalReplicaBytes.getAndAdd(bytes); + + memoryManager.decreaseShardReplicaLimits(tracker); + shardIndexingPressureStore.tryIndexingPressureTrackerCleanup(tracker); + }; + } + + return () -> { + long requestEndTime = System.currentTimeMillis(); + long requestLatency = requestEndTime - requestStartTime; + + indexingPressure.addAndGetCurrentReplicaBytes(-bytes); + tracker.currentReplicaBytes.addAndGet(-bytes); + tracker.replicaTimeInMillis.addAndGet(requestLatency); + tracker.totalReplicaBytes.getAndAdd(bytes); + tracker.lastSuccessfulReplicaRequestTimestamp.set(requestEndTime); + tracker.totalOutstandingReplicaRequests.set(0); + + if(requestLatency > 0) { + double requestThroughput = (double) bytes / requestLatency; + tracker.replicaThroughputMovingQueue.offer(requestThroughput); + if (tracker.replicaThroughputMovingQueue.size() > shardIndexingPressureSettings.getRequestSizeWindow()) { + double front = tracker.replicaThroughputMovingQueue.poll(); + double movingAverage = calculateMovingAverage(tracker.replicaThroughputMovingAverage.get(), front, + requestThroughput, shardIndexingPressureSettings.getRequestSizeWindow()); + tracker.replicaThroughputMovingAverage.set(Double.doubleToLongBits(movingAverage)); + } else { + double movingAverage = (double) tracker.totalReplicaBytes.get() / tracker.replicaTimeInMillis.get(); + tracker.replicaThroughputMovingAverage.set(Double.doubleToLongBits(movingAverage)); + } + } + memoryManager.decreaseShardReplicaLimits(tracker); + shardIndexingPressureStore.tryIndexingPressureTrackerCleanup(tracker); + }; + } + + private double calculateMovingAverage(long currentAverage, double frontValue, double currentValue, int count) { + if(count > 0) { + return ((Double.longBitsToDouble(currentAverage) * count) + currentValue - frontValue) / count; + } else { + return currentValue; + } + } + + public ShardIndexingPressureStats stats(CommonStatsFlags statsFlags) { + + if (statsFlags.includeOnlyTopIndexingPressureMetrics()) { + return topStats(); + } else { + ShardIndexingPressureStats allStats = stats(); + if (statsFlags.includeAllShardIndexingPressureTrackers()) { + allStats.addAll(coldStats()); + } + return allStats; + } + } + + ShardIndexingPressureStats stats() { + Map statsPerShard = new HashMap<>(); + boolean isEnforcedMode = shardIndexingPressureSettings.isShardIndexingPressureEnforced(); + + for (Map.Entry shardEntry : + this.shardIndexingPressureStore.getShardIndexingPressureHotStore().entrySet()) { + IndexingPressurePerShardStats shardStats = new IndexingPressurePerShardStats(shardEntry.getValue(), + isEnforcedMode); + statsPerShard.put(shardEntry.getKey(), shardStats); + } + return new ShardIndexingPressureStats(statsPerShard, memoryManager.totalNodeLimitsBreachedRejections.get(), + memoryManager.totalLastSuccessfulRequestLimitsBreachedRejections.get(), + memoryManager.totalThroughputDegradationLimitsBreachedRejections.get(), + shardIndexingPressureSettings.isShardIndexingPressureEnabled(), + isEnforcedMode); + } + + ShardIndexingPressureStats coldStats() { + Map statsPerShard = new HashMap<>(); + boolean isEnforcedMode = shardIndexingPressureSettings.isShardIndexingPressureEnforced(); + + for (Map.Entry shardEntry : + this.shardIndexingPressureStore.getShardIndexingPressureColdStore().entrySet()) { + IndexingPressurePerShardStats shardStats = new IndexingPressurePerShardStats(shardEntry.getValue(), + isEnforcedMode); + statsPerShard.put(shardEntry.getKey(), shardStats); + } + return new ShardIndexingPressureStats(statsPerShard, memoryManager.totalNodeLimitsBreachedRejections.get(), + memoryManager.totalLastSuccessfulRequestLimitsBreachedRejections.get(), + memoryManager.totalThroughputDegradationLimitsBreachedRejections.get(), + shardIndexingPressureSettings.isShardIndexingPressureEnabled(), + isEnforcedMode); + } + + ShardIndexingPressureStats topStats() { + return new ShardIndexingPressureStats(Collections.emptyMap(), memoryManager.totalNodeLimitsBreachedRejections.get(), + memoryManager.totalLastSuccessfulRequestLimitsBreachedRejections.get(), + memoryManager.totalThroughputDegradationLimitsBreachedRejections.get(), + shardIndexingPressureSettings.isShardIndexingPressureEnabled(), + shardIndexingPressureSettings.isShardIndexingPressureEnforced()); + } + + ShardIndexingPressureTracker getShardIndexingPressureTracker(ShardId shardId) { + return shardIndexingPressureStore.getShardIndexingPressureTracker(shardId); + } + + public static boolean isShardIndexingPressureAttributeEnabled() { + Iterator nodes = clusterService.state().getNodes().getNodes().valuesIt(); + while (nodes.hasNext()) { + if (!Boolean.parseBoolean(nodes.next().getAttributes().get(SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY))) { + return false; + } + } + return true; + } + + public boolean isShardIndexingPressureEnabled() { + return shardIndexingPressureSettings.isShardIndexingPressureEnabled(); + } +} diff --git a/server/src/main/java/org/opensearch/index/ShardIndexingPressureMemoryManager.java b/server/src/main/java/org/opensearch/index/ShardIndexingPressureMemoryManager.java new file mode 100644 index 0000000000000..9c0c65665db38 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/ShardIndexingPressureMemoryManager.java @@ -0,0 +1,540 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Setting.Property; +import org.opensearch.common.settings.Settings; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +/** + * The Shard Indexing Pressure Memory Manager is the class which will be responsible for increasing and decreasing the + * limits given to a shard in a thread safe manner. The limits is the maximum space that a shard can occupy in the heap + * and the values will be modified in certain scenarios. + * + * 1. If the limits assigned to the shard is breached(Primary Parameter) and the node level occupancy of all shards + * is not greater than 70%(Primary Parameter), we will be increasing the shard limits without any further evaluation. + * 2. If the limits assigned to the shard is breached(Primary Parameter) and the node level occupancy of all the shards + * is greater than 70%(Primary Parameter) is when we will evaluate certain parameters like throughput degradation(Secondary Parameter) + * and last successful request elapsed timeout(Secondary Parameter) to evaluate if the limits for the shard needs to + * be modified or not. + * + * Secondary Parameters + * 1. ThroughputDegradationLimitsBreached - When the moving window throughput average has increased by some factor than + * the historical throughput average. If the factor by which it has increased is greater than the degradation limit this + * parameter is said to be breached. + * 2. LastSuccessfulRequestDurationLimitsBreached - When the difference between last successful request timestamp and + * current request timestamp is greater than the max timeout value and the number of outstanding requests is greater + * than the max outstanding requests then this parameter is said to be breached. + * + * Note : Every time we try to increase of decrease the shard limits. In case the shard utilization goes below 75% or + * goes above 95% of current shard limits then we try to set the new shard limit to be 85% of + * current shard utilization. + * + */ +public class ShardIndexingPressureMemoryManager { + private final Logger logger = LogManager.getLogger(getClass()); + + /* + Operating factor can be evaluated using currentShardBytes/shardLimits. Outcome of this expression is categorized as + lower, optimal and upper and appropriate action is taken once they breach the value mentioned below. + */ + public static final Setting LOWER_OPERATING_FACTOR = + Setting.doubleSetting("aes.shard_indexing_pressure.operating_factor.lower", 0.75d, 0.0d, Property.NodeScope, Property.Dynamic); + public static final Setting OPTIMAL_OPERATING_FACTOR = + Setting.doubleSetting("aes.shard_indexing_pressure.operating_factor.optimal", 0.85d, 0.0d, Property.NodeScope, Property.Dynamic); + public static final Setting UPPER_OPERATING_FACTOR = + Setting.doubleSetting("aes.shard_indexing_pressure.operating_factor.upper", 0.95d, 0.0d, Property.NodeScope, Property.Dynamic); + + /* + This is the max time that can be elapsed after any request is processed successfully. Appropriate action is taken + once the below mentioned value is breached. + */ + public static final Setting SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT = + Setting.intSetting("aes.shard_indexing_pressure.secondary_parameter.successful_request.elapsed_timeout", 300000, + Property.NodeScope, Property.Dynamic); + + /* + This is the max outstanding request that are present after any request is processed successfully. Appropriate + action is taken once the below mentioned value is breached. + */ + public static final Setting MAX_OUTSTANDING_REQUESTS = + Setting.intSetting("aes.shard_indexing_pressure.secondary_parameter.successful_request.max_outstanding_requests", + 100, Property.NodeScope, Property.Dynamic); + + /* + Degradation limits can be evaluated using average throughput last N requests + and N being {@link ShardIndexingPressure#WINDOW_SIZE} divided by lifetime average throughput. + Appropriate action is taken once the outcome of above expression breaches the below mentioned factor + */ + public static final Setting THROUGHPUT_DEGRADATION_LIMITS = + Setting.doubleSetting("aes.shard_indexing_pressure.secondary_parameter.throughput.degradation_factor", 5.0d, 1.0d, + Property.NodeScope, Property.Dynamic); + + /* + The secondary parameter accounting factor tells when the secondary parameter is considered. i.e. If the current + node level memory utilization divided by the node limits is greater than 70% then appropriate action is taken. + */ + public static final Setting NODE_SOFT_LIMIT = + Setting.doubleSetting("aes.shard_indexing_pressure.primary_parameter.node.soft_limit", 0.7d, 0.0d, + Property.NodeScope, Property.Dynamic); + + public final AtomicLong totalNodeLimitsBreachedRejections = new AtomicLong(); + public final AtomicLong totalLastSuccessfulRequestLimitsBreachedRejections = new AtomicLong(); + public final AtomicLong totalThroughputDegradationLimitsBreachedRejections = new AtomicLong(); + + private final ShardIndexingPressureSettings shardIndexingPressureSettings; + + private volatile double lowerOperatingFactor; + private volatile double optimalOperatingFactor; + private volatile double upperOperatingFactor; + + private volatile int successfulRequestElapsedTimeout; + private volatile int maxOutstandingRequests; + + private volatile double primaryAndCoordinatingThroughputDegradationLimits; + private volatile double replicaThroughputDegradationLimits; + + private volatile double nodeSoftLimit; + + public ShardIndexingPressureMemoryManager(ShardIndexingPressureSettings shardIndexingPressureSettings, + ClusterSettings clusterSettings, Settings settings) { + this.shardIndexingPressureSettings = shardIndexingPressureSettings; + + this.lowerOperatingFactor = LOWER_OPERATING_FACTOR.get(settings).doubleValue(); + clusterSettings.addSettingsUpdateConsumer(LOWER_OPERATING_FACTOR, this::setLowerOperatingFactor); + + this.optimalOperatingFactor = OPTIMAL_OPERATING_FACTOR.get(settings).doubleValue(); + clusterSettings.addSettingsUpdateConsumer(OPTIMAL_OPERATING_FACTOR, this::setOptimalOperatingFactor); + + this.upperOperatingFactor = UPPER_OPERATING_FACTOR.get(settings).doubleValue(); + clusterSettings.addSettingsUpdateConsumer(UPPER_OPERATING_FACTOR, this::setUpperOperatingFactor); + + this.successfulRequestElapsedTimeout = SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.get(settings).intValue(); + clusterSettings.addSettingsUpdateConsumer(SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT, this::setSuccessfulRequestElapsedTimeout); + + this.maxOutstandingRequests = MAX_OUTSTANDING_REQUESTS.get(settings).intValue(); + clusterSettings.addSettingsUpdateConsumer(MAX_OUTSTANDING_REQUESTS, this::setMaxOutstandingRequests); + + this.primaryAndCoordinatingThroughputDegradationLimits = THROUGHPUT_DEGRADATION_LIMITS.get(settings).doubleValue(); + this.replicaThroughputDegradationLimits = this.primaryAndCoordinatingThroughputDegradationLimits * 1.5; + clusterSettings.addSettingsUpdateConsumer(THROUGHPUT_DEGRADATION_LIMITS, this::setThroughputDegradationLimits); + + this.nodeSoftLimit = NODE_SOFT_LIMIT.get(settings).doubleValue(); + clusterSettings.addSettingsUpdateConsumer(NODE_SOFT_LIMIT, this::setNodeSoftLimit); + } + + boolean isPrimaryNodeLimitBreached(ShardIndexingPressureTracker tracker, long nodeTotalBytes) { + + //Checks if the node level threshold is breached. + if(nodeTotalBytes > this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits()) { + logger.debug("Node limits breached for primary operation [node_total_bytes={}, " + + "node_primary_and_coordinating_limits={}]", nodeTotalBytes, + this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits()); + tracker.primaryNodeLimitsBreachedRejections.incrementAndGet(); + totalNodeLimitsBreachedRejections.incrementAndGet(); + + return true; + } + return false; + } + + boolean isPrimaryShardLimitBreached(ShardIndexingPressureTracker tracker, long requestStartTime, + Map shardIndexingPressureStore, long nodeTotalBytes) { + + //Memory limits is breached when the current utilization is greater than 95% of total shard limits. + long shardCombinedBytes = tracker.currentCombinedCoordinatingAndPrimaryBytes.get(); + long shardPrimaryAndCoordinatingLimits = tracker.primaryAndCoordinatingLimits.get(); + boolean shardMemoryLimitsBreached = + ((double)shardCombinedBytes / shardPrimaryAndCoordinatingLimits) > this.upperOperatingFactor; + + if(shardMemoryLimitsBreached) { + /* + Secondary Parameters(i.e. LastSuccessfulRequestDuration and Throughput) is taken into consideration when + the current node utilization is greater than 70% of total node limits. + */ + if(((double)nodeTotalBytes / this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits()) < this.nodeSoftLimit) { + boolean isShardLimitsIncreased = + this.increaseShardPrimaryAndCoordinatingLimits(tracker, shardIndexingPressureStore); + if(!isShardLimitsIncreased) { + tracker.primaryNodeLimitsBreachedRejections.incrementAndGet(); + totalNodeLimitsBreachedRejections.incrementAndGet(); + } + + return !isShardLimitsIncreased; + } else { + boolean shardLastSuccessfulRequestDurationLimitsBreached = + this.evaluateLastSuccessfulRequestDurationLimitsBreached(tracker.lastSuccessfulPrimaryRequestTimestamp.get(), + requestStartTime, tracker.totalOutstandingPrimaryRequests.get()); + + boolean shardThroughputDegradationLimitsBreached = + this.evaluateThroughputDegradationLimitsBreached( + Double.longBitsToDouble(tracker.primaryThroughputMovingAverage.get()), + tracker.totalPrimaryBytes.get(), tracker.primaryTimeInMillis.get(), + tracker.primaryThroughputMovingQueue.size(), primaryAndCoordinatingThroughputDegradationLimits); + + if(shardLastSuccessfulRequestDurationLimitsBreached || shardThroughputDegradationLimitsBreached) { + if(shardLastSuccessfulRequestDurationLimitsBreached) { + tracker.primaryLastSuccessfulRequestLimitsBreachedRejections.incrementAndGet(); + totalLastSuccessfulRequestLimitsBreachedRejections.incrementAndGet(); + } else if(shardThroughputDegradationLimitsBreached) { + tracker.primaryThroughputDegradationLimitsBreachedRejections.incrementAndGet(); + totalThroughputDegradationLimitsBreachedRejections.incrementAndGet(); + } + + return true; + } else { + boolean isShardLimitsIncreased = + this.increaseShardPrimaryAndCoordinatingLimits(tracker, shardIndexingPressureStore); + if(!isShardLimitsIncreased) { + tracker.primaryNodeLimitsBreachedRejections.incrementAndGet(); + totalNodeLimitsBreachedRejections.incrementAndGet(); + } + + return !isShardLimitsIncreased; + } + } + } else { + return false; + } + } + + boolean isCoordinatingNodeLimitBreached(ShardIndexingPressureTracker tracker, long nodeTotalBytes) { + + //Checks if the node level threshold is breached. + if(nodeTotalBytes > this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits()) { + logger.debug("Node limits breached for coordinating operation [node_total_bytes={} , " + + "node_primary_and_coordinating_limits={}]", nodeTotalBytes, + this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits()); + tracker.coordinatingNodeLimitsBreachedRejections.incrementAndGet(); + totalNodeLimitsBreachedRejections.incrementAndGet(); + + return true; + } + return false; + } + + boolean isCoordinatingShardLimitBreached(ShardIndexingPressureTracker tracker, long requestStartTime, + Map shardIndexingPressureStore, long nodeTotalBytes) { + + //Shard memory limit is breached when the current utilization is greater than 95% of total shard limits. + long shardCombinedBytes = tracker.currentCombinedCoordinatingAndPrimaryBytes.get(); + long shardPrimaryAndCoordinatingLimits = tracker.primaryAndCoordinatingLimits.get(); + boolean shardMemoryLimitsBreached = + ((double)shardCombinedBytes / shardPrimaryAndCoordinatingLimits) > this.upperOperatingFactor; + + if(shardMemoryLimitsBreached) { + /* + Secondary Parameters(i.e. LastSuccessfulRequestDuration and Throughput) is taken into consideration when + the current node utilization is greater than 70% of total node limits. + */ + if(((double)nodeTotalBytes / this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits()) < this.nodeSoftLimit) { + boolean isShardLimitsIncreased = + this.increaseShardPrimaryAndCoordinatingLimits(tracker, shardIndexingPressureStore); + if(!isShardLimitsIncreased) { + tracker.coordinatingNodeLimitsBreachedRejections.incrementAndGet(); + totalNodeLimitsBreachedRejections.incrementAndGet(); + } + + return !isShardLimitsIncreased; + } else { + boolean shardLastSuccessfulRequestDurationLimitsBreached = + this.evaluateLastSuccessfulRequestDurationLimitsBreached(tracker.lastSuccessfulCoordinatingRequestTimestamp.get(), + requestStartTime, tracker.totalOutstandingCoordinatingRequests.get()); + + boolean shardThroughputDegradationLimitsBreached = + this.evaluateThroughputDegradationLimitsBreached( + Double.longBitsToDouble(tracker.coordinatingThroughputMovingAverage.get()), + tracker.totalCoordinatingBytes.get(), tracker.coordinatingTimeInMillis.get(), + tracker.coordinatingThroughputMovingQueue.size(), primaryAndCoordinatingThroughputDegradationLimits); + + if (shardLastSuccessfulRequestDurationLimitsBreached || shardThroughputDegradationLimitsBreached) { + if(shardLastSuccessfulRequestDurationLimitsBreached) { + tracker.coordinatingLastSuccessfulRequestLimitsBreachedRejections.incrementAndGet(); + totalLastSuccessfulRequestLimitsBreachedRejections.incrementAndGet(); + } else if(shardThroughputDegradationLimitsBreached) { + tracker.coordinatingThroughputDegradationLimitsBreachedRejections.incrementAndGet(); + totalThroughputDegradationLimitsBreachedRejections.incrementAndGet(); + } + + return true; + } else { + boolean isShardLimitsIncreased = + this.increaseShardPrimaryAndCoordinatingLimits(tracker, shardIndexingPressureStore); + if(!isShardLimitsIncreased) { + tracker.coordinatingNodeLimitsBreachedRejections.incrementAndGet(); + totalNodeLimitsBreachedRejections.incrementAndGet(); + } + + return !isShardLimitsIncreased; + } + } + } else { + return false; + } + } + + boolean isReplicaNodeLimitBreached(ShardIndexingPressureTracker tracker, long nodeReplicaBytes) { + + //Checks if the node level threshold is breached. + if(nodeReplicaBytes > this.shardIndexingPressureSettings.getNodeReplicaLimits()) { + logger.debug("Node limits breached for replica operation [node_replica_bytes={} , " + + "node_replica_limits={}]", nodeReplicaBytes, this.shardIndexingPressureSettings.getNodeReplicaLimits()); + tracker.replicaNodeLimitsBreachedRejections.incrementAndGet(); + totalNodeLimitsBreachedRejections.incrementAndGet(); + + return true; + } + return false; + } + + boolean isReplicaShardLimitBreached(ShardIndexingPressureTracker tracker, long requestStartTime, + Map shardIndexingPressureStore, long nodeReplicaBytes) { + + //Memory limits is breached when the current utilization is greater than 95% of total shard limits. + long shardReplicaBytes = tracker.currentReplicaBytes.get(); + long shardReplicaLimits = tracker.replicaLimits.get(); + final boolean shardMemoryLimitsBreached = + ((double)shardReplicaBytes / shardReplicaLimits) > this.upperOperatingFactor; + + if(shardMemoryLimitsBreached) { + /* + Secondary Parameters(i.e. LastSuccessfulRequestDuration and Throughput) is taken into consideration when + the current node utilization is greater than 70% of total node limits. + */ + if(((double)nodeReplicaBytes / this.shardIndexingPressureSettings.getNodeReplicaLimits()) < this.nodeSoftLimit) { + boolean isShardLimitsIncreased = + this.increaseShardReplicaLimits(tracker, shardIndexingPressureStore); + if(!isShardLimitsIncreased) { + tracker.replicaNodeLimitsBreachedRejections.incrementAndGet(); + totalNodeLimitsBreachedRejections.incrementAndGet(); + } + + return !isShardLimitsIncreased; + } else { + boolean shardLastSuccessfulRequestDurationLimitsBreached = + this.evaluateLastSuccessfulRequestDurationLimitsBreached(tracker.lastSuccessfulReplicaRequestTimestamp.get(), + requestStartTime, tracker.totalOutstandingReplicaRequests.get()); + + boolean shardThroughputDegradationLimitsBreached = + this.evaluateThroughputDegradationLimitsBreached( + Double.longBitsToDouble(tracker.replicaThroughputMovingAverage.get()), + tracker.totalReplicaBytes.get(), tracker.replicaTimeInMillis.get(), + tracker.replicaThroughputMovingQueue.size(), replicaThroughputDegradationLimits); + + if (shardLastSuccessfulRequestDurationLimitsBreached || shardThroughputDegradationLimitsBreached) { + if(shardLastSuccessfulRequestDurationLimitsBreached) { + tracker.replicaLastSuccessfulRequestLimitsBreachedRejections.incrementAndGet(); + totalLastSuccessfulRequestLimitsBreachedRejections.incrementAndGet(); + } else if(shardThroughputDegradationLimitsBreached) { + tracker.replicaThroughputDegradationLimitsBreachedRejections.incrementAndGet(); + totalThroughputDegradationLimitsBreachedRejections.incrementAndGet(); + } + + return true; + } else { + boolean isShardLimitsIncreased = + this.increaseShardReplicaLimits(tracker, shardIndexingPressureStore); + if(!isShardLimitsIncreased) { + tracker.replicaNodeLimitsBreachedRejections.incrementAndGet(); + totalNodeLimitsBreachedRejections.incrementAndGet(); + } + + return !isShardLimitsIncreased; + } + } + } else { + return false; + } + } + + private boolean increaseShardPrimaryAndCoordinatingLimits(ShardIndexingPressureTracker tracker, + Map shardIndexingPressureStore) { + long shardPrimaryAndCoordinatingLimits; + long expectedShardPrimaryAndCoordinatingLimits; + do { + shardPrimaryAndCoordinatingLimits = tracker.primaryAndCoordinatingLimits.get(); + long shardCombinedBytes = tracker.currentCombinedCoordinatingAndPrimaryBytes.get(); + expectedShardPrimaryAndCoordinatingLimits = (long)(shardCombinedBytes / this.optimalOperatingFactor); + + long totalPrimaryAndCoordinatingLimitsExceptCurrentShard = shardIndexingPressureStore.entrySet().stream() + .filter(entry -> !(tracker.shardId.hashCode() == entry.getKey())) + .map(Map.Entry::getValue) + .mapToLong(entry -> entry.primaryAndCoordinatingLimits.get()).sum(); + + if(((double)shardCombinedBytes / shardPrimaryAndCoordinatingLimits) > this.upperOperatingFactor) { + if (totalPrimaryAndCoordinatingLimitsExceptCurrentShard + expectedShardPrimaryAndCoordinatingLimits < + this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits()) { + logger.debug("Increasing the Primary And Coordinating Limits [" + + "shard_detail=[{}][{}], shard_max_primary_and_coordinating_bytes={}, " + + "expected_shard_max_primary_and_coordinating_bytes={}]", + tracker.shardId.getIndexName(), tracker.shardId.id(), + shardPrimaryAndCoordinatingLimits, expectedShardPrimaryAndCoordinatingLimits); + } else { + logger.debug("Failed to increase the Primary And Coordinating Limits [shard_detail=[{}][{}}], " + + "shard_max_primary_and_coordinating_bytes={}, " + + "total_max_primary_and_coordinating_bytes_except_current_shard={}, " + + "expected_shard_max_primary_and_coordinating_bytes={}, node_max_coordinating_and_primary_bytes={}]", + tracker.shardId.getIndexName(), tracker.shardId.id(), shardPrimaryAndCoordinatingLimits, + totalPrimaryAndCoordinatingLimitsExceptCurrentShard, expectedShardPrimaryAndCoordinatingLimits, + this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits()); + return false; + } + } else { + return true; + } + } while(!tracker.primaryAndCoordinatingLimits.compareAndSet(shardPrimaryAndCoordinatingLimits, + expectedShardPrimaryAndCoordinatingLimits)); + return true; + } + + void decreaseShardPrimaryAndCoordinatingLimits(ShardIndexingPressureTracker tracker) { + long shardPrimaryAndCoordinatingLimits; + long expectedShardPrimaryAndCoordinatingLimits; + do { + shardPrimaryAndCoordinatingLimits = tracker.primaryAndCoordinatingLimits.get(); + long shardCombinedBytes = tracker.currentCombinedCoordinatingAndPrimaryBytes.get(); + expectedShardPrimaryAndCoordinatingLimits = Math.max((long) (shardCombinedBytes / this.optimalOperatingFactor), + this.shardIndexingPressureSettings.getShardPrimaryAndCoordinatingBaseLimits()); + + if (((double)shardCombinedBytes / shardPrimaryAndCoordinatingLimits) < this.lowerOperatingFactor) { + logger.debug("Decreasing the Primary And Coordinating Limits [shard_detail=[{}][{}], " + + "shard_max_primary_and_coordinating_bytes={}, expected_shard_max_primary_and_coordinating_bytes={}]", + tracker.shardId.getIndexName(), tracker.shardId.id(), + shardPrimaryAndCoordinatingLimits, expectedShardPrimaryAndCoordinatingLimits); + } else { + logger.debug("Primary And Coordinating Limits Already Increased [" + + "shard_detail=[{}][{}], " + "shard_max_primary_and_coordinating_bytes={}, " + + "expected_shard_max_primary_and_coordinating_bytes={}]", + tracker.shardId.getIndexName(), tracker.shardId.id(), shardPrimaryAndCoordinatingLimits, + expectedShardPrimaryAndCoordinatingLimits); + return; + } + } while(!tracker.primaryAndCoordinatingLimits.compareAndSet(shardPrimaryAndCoordinatingLimits, + expectedShardPrimaryAndCoordinatingLimits)); + } + + private boolean increaseShardReplicaLimits(ShardIndexingPressureTracker tracker, + Map shardIndexingPressureStore) { + long shardReplicaLimits; + long expectedShardReplicaLimits; + do { + shardReplicaLimits = tracker.replicaLimits.get(); + long shardReplicaBytes = tracker.currentReplicaBytes.get(); + expectedShardReplicaLimits = (long)(shardReplicaBytes / this.optimalOperatingFactor); + + long totalReplicaLimitsExceptCurrentShard = shardIndexingPressureStore.entrySet().stream() + .filter(entry -> !(tracker.shardId.hashCode() == entry.getKey())) + .map(Map.Entry::getValue) + .mapToLong(entry -> entry.replicaLimits.get()).sum(); + + if(((double)shardReplicaBytes / shardReplicaLimits) > this.upperOperatingFactor) { + if (totalReplicaLimitsExceptCurrentShard + expectedShardReplicaLimits < + this.shardIndexingPressureSettings.getNodeReplicaLimits()) { + logger.debug("Increasing the Replica Limits [shard_detail=[{}][{}], " + + "shard_max_replica_bytes={}, expected_shard_max_replica_bytes={}]", + tracker.shardId.getIndexName(), tracker.shardId.id(), + shardReplicaLimits, expectedShardReplicaLimits); + } else { + logger.debug("Failed to increase the Replica Limits [shard_detail=[{}][{}], " + + "shard_max_replica_bytes={}, total_max_replica_except_current_shard={}}, " + + "expected_shard_max_replica_bytes={}, node_max_replica_bytes={}]", + tracker.shardId.getIndexName(), tracker.shardId.id(), shardReplicaLimits, + totalReplicaLimitsExceptCurrentShard, expectedShardReplicaLimits, + this.shardIndexingPressureSettings.getNodeReplicaLimits()); + return false; + } + } else { + return true; + } + } while(!tracker.replicaLimits.compareAndSet(shardReplicaLimits, expectedShardReplicaLimits)); + return true; + } + + void decreaseShardReplicaLimits(ShardIndexingPressureTracker tracker) { + + long shardReplicaLimits; + long expectedShardReplicaLimits; + do { + shardReplicaLimits = tracker.replicaLimits.get(); + long shardReplicaBytes = tracker.currentReplicaBytes.get(); + expectedShardReplicaLimits = Math.max((long) (shardReplicaBytes / this.optimalOperatingFactor), + this.shardIndexingPressureSettings.getShardReplicaBaseLimits()); + + if (((double)shardReplicaBytes / shardReplicaLimits) < this.lowerOperatingFactor) { + logger.debug("Decreasing the Replica Limits [shard_detail=[{}}][{}}], " + + "shard_max_replica_bytes={}, expected_shard_max_replica_bytes={}]", + tracker.shardId.getIndexName(), tracker.shardId.id(), shardReplicaLimits, + expectedShardReplicaLimits); + } else { + logger.debug("Replica Limits Already Increased [shard_detail=[{}][{}], " + + "shard_max_replica_bytes={}, expected_shard_max_replica_bytes={}]", + tracker.shardId.getIndexName(), tracker.shardId.id(), shardReplicaLimits, + expectedShardReplicaLimits); + return; + } + } while(!tracker.replicaLimits.compareAndSet(shardReplicaLimits, expectedShardReplicaLimits)); + } + + /** + * Throughput of last N request divided by the total lifetime requests throughput is greater than the acceptable + * degradation limits then we say this parameter has breached the threshold. + */ + private boolean evaluateThroughputDegradationLimitsBreached(double throughputMovingAverage, + long totalBytes, long totalLatency, + long queueSize, double degradationLimits) { + double throughputHistoricalAverage = (double)totalBytes / totalLatency; + return throughputMovingAverage > 0 && queueSize >= shardIndexingPressureSettings.getRequestSizeWindow() + && throughputHistoricalAverage / throughputMovingAverage > degradationLimits; + } + + /** + * The difference in the current timestamp and last successful request timestamp is greater than + * successful request elapsed timeout value and the total number of outstanding requests is greater than + * the maximum outstanding request count value then we say this parameter has breached the threshold. + */ + private boolean evaluateLastSuccessfulRequestDurationLimitsBreached(long lastSuccessfulRequestTimestamp, + long requestStartTime, + long totalOutstandingRequests) { + return (lastSuccessfulRequestTimestamp > 0) && + (((requestStartTime - lastSuccessfulRequestTimestamp) > this.successfulRequestElapsedTimeout + && totalOutstandingRequests > this.maxOutstandingRequests)); + } + + private void setLowerOperatingFactor(double lowerOperatingFactor) { + this.lowerOperatingFactor = lowerOperatingFactor; + } + + private void setOptimalOperatingFactor(double optimalOperatingFactor) { + this.optimalOperatingFactor = optimalOperatingFactor; + } + + private void setUpperOperatingFactor(double upperOperatingFactor) { + this.upperOperatingFactor = upperOperatingFactor; + } + + private void setSuccessfulRequestElapsedTimeout(int successfulRequestElapsedTimeout) { + this.successfulRequestElapsedTimeout = successfulRequestElapsedTimeout; + } + + private void setMaxOutstandingRequests(int maxOutstandingRequests) { + this.maxOutstandingRequests = maxOutstandingRequests; + } + + private void setThroughputDegradationLimits(double throughputDegradationLimits) { + this.primaryAndCoordinatingThroughputDegradationLimits = throughputDegradationLimits; + this.replicaThroughputDegradationLimits = this.primaryAndCoordinatingThroughputDegradationLimits * 1.5; + } + + private void setNodeSoftLimit(double nodeSoftLimit) { + this.nodeSoftLimit = nodeSoftLimit; + } +} diff --git a/server/src/main/java/org/opensearch/index/ShardIndexingPressureSettings.java b/server/src/main/java/org/opensearch/index/ShardIndexingPressureSettings.java new file mode 100644 index 0000000000000..cf42c8616fb5b --- /dev/null +++ b/server/src/main/java/org/opensearch/index/ShardIndexingPressureSettings.java @@ -0,0 +1,120 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; + +/** + * This class contains all the setting which whose owner class in ShardIndexingPressure and it will be used in + * ShardIndexingPressure as well as the classes whose instantiation is done in ShardIndexingPressure, i.e. + * ShardIndexingPressureMemoryManager and ShardIndexingPressureStore + */ +public final class ShardIndexingPressureSettings { + + public static final Setting SHARD_INDEXING_PRESSURE_ENABLED = + Setting.boolSetting("aes.shard_indexing_pressure.enabled", false, Setting.Property.Dynamic, Setting.Property.NodeScope); + + /** + * Feature level setting to operate in shadow-mode or in enforced-mode. If enforced field is set to true, shard level + * rejection will be performed, otherwise only rejection metrics will be populated. + */ + public static final Setting SHARD_INDEXING_PRESSURE_ENFORCED = + Setting.boolSetting("aes.shard_indexing_pressure.enforced", false, Setting.Property.Dynamic, Setting.Property.NodeScope); + + // This represents the last N request samples that will be considered for secondary parameter evaluation. + public static final Setting REQUEST_SIZE_WINDOW = + Setting.intSetting("aes.shard_indexing_pressure.secondary_parameter.throughput.request_size_window", 2000, + Setting.Property.NodeScope, Setting.Property.Dynamic); + + //Each shard will be initially given 1/1000th bytes of node limits. + public static final Setting SHARD_MIN_LIMIT = + Setting.doubleSetting("aes.shard_indexing_pressure.primary_parameter.shard.min_limit", 0.001d, 0.0d, + Setting.Property.NodeScope, Setting.Property.Dynamic); + + private volatile boolean shardIndexingPressureEnabled; + private volatile boolean shardIndexingPressureEnforced; + private volatile long shardPrimaryAndCoordinatingBaseLimits; + private volatile long shardReplicaBaseLimits; + private volatile int requestSizeWindow; + private volatile double shardMinLimit; + private final long primaryAndCoordinatingNodeLimits; + + public ShardIndexingPressureSettings(ClusterSettings clusterSettings, Settings settings, long primaryAndCoordinatingLimits) { + this.shardIndexingPressureEnabled = SHARD_INDEXING_PRESSURE_ENABLED.get(settings); + clusterSettings.addSettingsUpdateConsumer(SHARD_INDEXING_PRESSURE_ENABLED, this::setShardIndexingPressureEnabled); + + this.shardIndexingPressureEnforced = SHARD_INDEXING_PRESSURE_ENFORCED.get(settings); + clusterSettings.addSettingsUpdateConsumer(SHARD_INDEXING_PRESSURE_ENFORCED, this::setShardIndexingPressureEnforced); + + this.requestSizeWindow = REQUEST_SIZE_WINDOW.get(settings).intValue(); + clusterSettings.addSettingsUpdateConsumer(REQUEST_SIZE_WINDOW, this::setRequestSizeWindow); + + this.primaryAndCoordinatingNodeLimits = primaryAndCoordinatingLimits; + + this.shardMinLimit = SHARD_MIN_LIMIT.get(settings).floatValue(); + this.shardPrimaryAndCoordinatingBaseLimits = (long) (primaryAndCoordinatingLimits * shardMinLimit); + this.shardReplicaBaseLimits = (long) (shardPrimaryAndCoordinatingBaseLimits * 1.5); + clusterSettings.addSettingsUpdateConsumer(SHARD_MIN_LIMIT, this::setShardMinLimit); + } + + private void setShardIndexingPressureEnabled(Boolean shardIndexingPressureEnableValue) { + this.shardIndexingPressureEnabled = shardIndexingPressureEnableValue; + } + + private void setShardIndexingPressureEnforced(Boolean shardIndexingPressureEnforcedValue) { + this.shardIndexingPressureEnforced = shardIndexingPressureEnforcedValue; + } + + private void setRequestSizeWindow(int requestSizeWindow) { + this.requestSizeWindow = requestSizeWindow; + } + + private void setShardMinLimit(double shardMinLimit) { + this.shardMinLimit = shardMinLimit; + + //Updating the dependent value once when the dynamic settings update + this.setShardPrimaryAndCoordinatingBaseLimits(); + this.setShardReplicaBaseLimits(); + } + + private void setShardPrimaryAndCoordinatingBaseLimits() { + shardPrimaryAndCoordinatingBaseLimits = (long) (primaryAndCoordinatingNodeLimits * shardMinLimit); + } + + private void setShardReplicaBaseLimits() { + shardReplicaBaseLimits = (long) (shardPrimaryAndCoordinatingBaseLimits * 1.5); + } + + public boolean isShardIndexingPressureEnabled() { + return shardIndexingPressureEnabled; + } + + public boolean isShardIndexingPressureEnforced() { + return shardIndexingPressureEnforced; + } + + public int getRequestSizeWindow() { + return requestSizeWindow; + } + + public long getShardPrimaryAndCoordinatingBaseLimits() { + return shardPrimaryAndCoordinatingBaseLimits; + } + + public long getShardReplicaBaseLimits() { + return shardReplicaBaseLimits; + } + + public long getNodePrimaryAndCoordinatingLimits() { + return primaryAndCoordinatingNodeLimits; + } + + public long getNodeReplicaLimits() { + return (long) (primaryAndCoordinatingNodeLimits * 1.5); + } +} diff --git a/server/src/main/java/org/opensearch/index/ShardIndexingPressureStore.java b/server/src/main/java/org/opensearch/index/ShardIndexingPressureStore.java new file mode 100644 index 0000000000000..516f4fa4d9952 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/ShardIndexingPressureStore.java @@ -0,0 +1,122 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.index.shard.ShardId; + +import java.util.Collections; +import java.util.Map; + +import static java.util.Objects.isNull; + +/** + * The Shard indexing pressure store acts as a central repository for all the shard-level tracking objects being + * used at a node level in order to track indexing pressure. It manages the tracker lifecycle. + * + * The shardIndexingPressureHotStore is a primary (hot) store and holds all the shard tracking object which are + * currently live i.e. they are performing request level tracking for in-flight requests. + * + * The shardIndexingPressureColdStore acts as the cold storage for all the shard tracking objects which were created, + * but are not currently live i.e. they are not tracking any in-flight requests currently. + * + * Tracking objects when created are part of both the hot store as well as cold store. However, once the object + * is no more live it is removed from the hot store. Objects in the cold store are evicted once the cold store + * reaches its maximum limit. Think of it like a periodic archival purge. + * During get if tracking object is not present in the hot store, a lookup is made into the cache store. If found, + * object is brought into the hot store again, until it remains active. If not present in the either store, a fresh + * object is instantiated an registered in both the stores. + * + * Note: The implementation of shardIndexingPressureColdStore methods is such that get, + * update and evict are abstracted out such that LRU logic can be plugged into it, if discovered a need later. + */ +public class ShardIndexingPressureStore { + + // This represents the initial value of cold store size. + public static final Setting MAX_CACHE_STORE_SIZE = + Setting.intSetting("aes.shard_indexing_pressure.cache_store.max_size", 200, Setting.Property.NodeScope, Setting.Property.Dynamic); + + private final Map shardIndexingPressureHotStore = + ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(); + private final Map shardIndexingPressureColdStore = + ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(); + private final ShardIndexingPressureSettings shardIndexingPressureSettings; + + private int maxColdStoreSize; + + public ShardIndexingPressureStore(ShardIndexingPressureSettings shardIndexingPressureSettings, + ClusterSettings clusterSettings, Settings settings) { + this.shardIndexingPressureSettings = shardIndexingPressureSettings; + this.maxColdStoreSize = MAX_CACHE_STORE_SIZE.get(settings).intValue(); + clusterSettings.addSettingsUpdateConsumer(MAX_CACHE_STORE_SIZE, this::setMaxColdStoreSize); + } + + public ShardIndexingPressureTracker getShardIndexingPressureTracker(ShardId shardId) { + ShardIndexingPressureTracker tracker = shardIndexingPressureHotStore.get((long)shardId.hashCode()); + if (isNull(tracker)) { + // Attempt from Indexing pressure cold store + tracker = getIndexingPressureTrackerFromColdStore(shardId); + // If not present in cold store so instantiate a new one + if (isNull(tracker)) { + ShardIndexingPressureTracker newShardIndexingPressureTracker = new ShardIndexingPressureTracker(shardId); + newShardIndexingPressureTracker.primaryAndCoordinatingLimits.set(this.shardIndexingPressureSettings + .getShardPrimaryAndCoordinatingBaseLimits()); + newShardIndexingPressureTracker.replicaLimits.set(this.shardIndexingPressureSettings.getShardReplicaBaseLimits()); + // Try update the new shard stat to the hot store + tracker = shardIndexingPressureHotStore.putIfAbsent((long) shardId.hashCode(), newShardIndexingPressureTracker); + // Update the tracker so that we use the one actual in the hot store + tracker = tracker == null ? newShardIndexingPressureTracker : tracker; + // Write through into the cold store for future reference + updateIndexingPressureColdStore(tracker); + } else { + // Attempt update tracker to the primary store and return tracker finally in the store to avoid any race + ShardIndexingPressureTracker newTracker = shardIndexingPressureHotStore.putIfAbsent((long) shardId.hashCode(), tracker); + tracker = newTracker == null ? tracker : newTracker; + } + } + return tracker; + } + + public Map getShardIndexingPressureHotStore() { + return Collections.unmodifiableMap(shardIndexingPressureHotStore); + } + + public Map getShardIndexingPressureColdStore() { + return Collections.unmodifiableMap(shardIndexingPressureColdStore); + } + + public void tryIndexingPressureTrackerCleanup(ShardIndexingPressureTracker tracker) { + if (tracker.currentCombinedCoordinatingAndPrimaryBytes.get() == 0 && tracker.currentReplicaBytes.get() == 0) { + // Try inserting into cache again in case there was an eviction earlier + shardIndexingPressureColdStore.putIfAbsent((long)tracker.shardId.hashCode(), tracker); + // Remove from the active store + shardIndexingPressureHotStore.remove((long)tracker.shardId.hashCode(), tracker); + } + } + + private ShardIndexingPressureTracker getIndexingPressureTrackerFromColdStore(ShardId shardId) { + return shardIndexingPressureColdStore.get((long)shardId.hashCode()); + } + + private void updateIndexingPressureColdStore(ShardIndexingPressureTracker tracker) { + if (shardIndexingPressureColdStore.size() > maxColdStoreSize) { + evictColdStore(); + } + shardIndexingPressureColdStore.put((long)tracker.shardId.hashCode(), tracker); + } + + private void evictColdStore() { + shardIndexingPressureColdStore.clear(); + } + + private void setMaxColdStoreSize(int maxColdStoreSize) { + this.maxColdStoreSize = maxColdStoreSize; + } + +} diff --git a/server/src/main/java/org/opensearch/index/ShardIndexingPressureTracker.java b/server/src/main/java/org/opensearch/index/ShardIndexingPressureTracker.java new file mode 100644 index 0000000000000..72d5ea80e6e18 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/ShardIndexingPressureTracker.java @@ -0,0 +1,115 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index; + +import org.opensearch.index.shard.ShardId; + +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicLong; + +/** + * This class contains all the tracking objects that will be maintained against a shard and will be used and modified + * while evaluating shard indexing pressure related information for a shard. + * + * This class tracks these parameters at coordinating, primary and replica indexing stage. + * 1. CurrentBytes - Bytes of data that is inflight/processing for a shard. + * 2. TotalBytes - Total bytes that was processed successfully for a shard. + * 3. Counts - Total number of requests that were processed successfully for a shard. + * 4. TimeInMillis - Total indexing time take by requests that were processed successfully for a shard. + * 5. Rejections - Total number of requests that were rejected for a shard. + * 6. NodeLimitsBreachedRejections - Total number of requests that were rejected due to the node level limits breached. + * i.e. when a request for a shard came and there was no scope for the shard to grow as + * node level limit was already reached. + * 7. LastSuccessfulRequestLimitsBreachedRejections - Total number of requests that were rejected due to the + * last successful request limits breached for a shard. + * 8. ThroughputDegradationLimitsBreachedRejections - Total number of requests that were rejected due to the + * last successful request limits breached for a shard. + * 9. LastSuccessfulRequestTimestamp - Timestamp of last successful request for a shard. + * 10. TotalOutstandingRequests - At any given point how many requests are outstanding for a shard. + * 11. ThroughputMovingAverage - Hold the average throughput value for last N requests. + * 12. ThroughputMovingQueue - Queue that holds the last N requests throughput such that we have a sliding window + * which keeps moving everytime a new request comes such that at any given point we are looking + * at last N requests only. EWMA cannot be used here as it evaluate the historical average + * and here we need the average of just last N requests. + * + * For more details on 6,7,8,9,10,11 see {@link ShardIndexingPressureMemoryManager} + */ +public class ShardIndexingPressureTracker { + + //Memory trackers + public final AtomicLong currentCombinedCoordinatingAndPrimaryBytes = new AtomicLong(); + public final AtomicLong currentCoordinatingBytes = new AtomicLong(); + public final AtomicLong currentPrimaryBytes = new AtomicLong(); + public final AtomicLong currentReplicaBytes = new AtomicLong(); + + public final AtomicLong totalCombinedCoordinatingAndPrimaryBytes = new AtomicLong(); + public final AtomicLong totalCoordinatingBytes = new AtomicLong(); + public final AtomicLong totalPrimaryBytes = new AtomicLong(); + public final AtomicLong totalReplicaBytes = new AtomicLong(); + + //Count based trackers + public final AtomicLong coordinatingCount = new AtomicLong(); + public final AtomicLong primaryCount = new AtomicLong(); + public final AtomicLong replicaCount = new AtomicLong(); + + //Latency + public AtomicLong coordinatingTimeInMillis = new AtomicLong(); + public AtomicLong primaryTimeInMillis = new AtomicLong(); + public AtomicLong replicaTimeInMillis = new AtomicLong(); + + //Coordinating Rejection Count + public final AtomicLong coordinatingRejections = new AtomicLong(); + public final AtomicLong coordinatingNodeLimitsBreachedRejections = new AtomicLong(); + public final AtomicLong coordinatingLastSuccessfulRequestLimitsBreachedRejections = new AtomicLong(); + public final AtomicLong coordinatingThroughputDegradationLimitsBreachedRejections = new AtomicLong(); + + //Primary Rejection Count + public final AtomicLong primaryRejections = new AtomicLong(); + public final AtomicLong primaryNodeLimitsBreachedRejections = new AtomicLong(); + public final AtomicLong primaryLastSuccessfulRequestLimitsBreachedRejections = new AtomicLong(); + public final AtomicLong primaryThroughputDegradationLimitsBreachedRejections = new AtomicLong(); + + //Replica Rejection Count + public final AtomicLong replicaRejections = new AtomicLong(); + public final AtomicLong replicaNodeLimitsBreachedRejections = new AtomicLong(); + public final AtomicLong replicaLastSuccessfulRequestLimitsBreachedRejections = new AtomicLong(); + public final AtomicLong replicaThroughputDegradationLimitsBreachedRejections = new AtomicLong(); + + //Last Successful TimeStamp + public final AtomicLong lastSuccessfulCoordinatingRequestTimestamp = new AtomicLong(); + public final AtomicLong lastSuccessfulPrimaryRequestTimestamp = new AtomicLong(); + public final AtomicLong lastSuccessfulReplicaRequestTimestamp = new AtomicLong(); + + //Total Outstanding requests after last successful request + public final AtomicLong totalOutstandingCoordinatingRequests = new AtomicLong(); + public final AtomicLong totalOutstandingPrimaryRequests = new AtomicLong(); + public final AtomicLong totalOutstandingReplicaRequests = new AtomicLong(); + + /* + Shard Window Throughput Tracker. + We will be using atomic long to track double values as mentioned here - + https://docs.oracle.com/javase/6/docs/api/java/util/concurrent/atomic/package-summary.html + */ + public final AtomicLong coordinatingThroughputMovingAverage = new AtomicLong(); + public final AtomicLong primaryThroughputMovingAverage = new AtomicLong(); + public final AtomicLong replicaThroughputMovingAverage = new AtomicLong(); + + //Shard Window Throughput Queue + public final ConcurrentLinkedQueue coordinatingThroughputMovingQueue = new ConcurrentLinkedQueue(); + public final ConcurrentLinkedQueue primaryThroughputMovingQueue = new ConcurrentLinkedQueue(); + public final ConcurrentLinkedQueue replicaThroughputMovingQueue = new ConcurrentLinkedQueue(); + + //Shard Reference + public final ShardId shardId; + + //Memory Allotted + public final AtomicLong primaryAndCoordinatingLimits = new AtomicLong(0); + public final AtomicLong replicaLimits = new AtomicLong(0); + + public ShardIndexingPressureTracker(ShardId shardId) { + this.shardId = shardId; + } +} diff --git a/server/src/main/java/org/opensearch/index/stats/IndexingPressurePerShardStats.java b/server/src/main/java/org/opensearch/index/stats/IndexingPressurePerShardStats.java new file mode 100644 index 0000000000000..a18f7313a3233 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/stats/IndexingPressurePerShardStats.java @@ -0,0 +1,410 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index.stats; + +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.common.xcontent.ToXContent; +import org.opensearch.common.xcontent.ToXContentFragment; +import org.opensearch.common.xcontent.XContentBuilder; +import org.opensearch.index.ShardIndexingPressureTracker; + +import java.io.IOException; + +public class IndexingPressurePerShardStats implements Writeable, ToXContentFragment { + + private final String shardId; + + private final long totalCombinedCoordinatingAndPrimaryBytes; + private final long totalCoordinatingBytes; + private final long totalPrimaryBytes; + private final long totalReplicaBytes; + + private final long currentCombinedCoordinatingAndPrimaryBytes; + private final long currentCoordinatingBytes; + private final long currentPrimaryBytes; + private final long currentReplicaBytes; + + private final long totalCoordinatingCount; + private final long totalPrimaryCount; + private final long totalReplicaCount; + + private final long coordinatingRejections; + private final long coordinatingNodeLimitsBreachedRejections; + private final long coordinatingLastSuccessfulRequestLimitsBreachedRejections; + private final long coordinatingThroughputDegradationLimitsBreachedRejections; + + private final long primaryRejections; + private final long primaryNodeLimitsBreachedRejections; + private final long primaryLastSuccessfulRequestLimitsBreachedRejections; + private final long primaryThroughputDegradationLimitsBreachedRejections; + + private final long replicaRejections; + private final long replicaNodeLimitsBreachedRejections; + private final long replicaLastSuccessfulRequestLimitsBreachedRejections; + private final long replicaThroughputDegradationLimitsBreachedRejections; + + private final long coordinatingTimeInMillis; + private final long primaryTimeInMillis; + private final long replicaTimeInMillis; + + private final long coordinatingLastSuccessfulRequestTimestampInMillis; + private final long primaryLastSuccessfulRequestTimestampInMillis; + private final long replicaLastSuccessfulRequestTimestampInMillis; + + private final long currentPrimaryAndCoordinatingLimits; + private final long currentReplicaLimits; + + private final boolean shardIndexingPressureEnforced; + + public IndexingPressurePerShardStats(StreamInput in) throws IOException { + shardId = in.readString(); + shardIndexingPressureEnforced = in.readBoolean(); + + totalCombinedCoordinatingAndPrimaryBytes = in.readVLong(); + totalCoordinatingBytes = in.readVLong(); + totalPrimaryBytes = in.readVLong(); + totalReplicaBytes = in.readVLong(); + + currentCombinedCoordinatingAndPrimaryBytes = in.readVLong(); + currentCoordinatingBytes = in.readVLong(); + currentPrimaryBytes = in.readVLong(); + currentReplicaBytes = in.readVLong(); + + totalCoordinatingCount = in.readVLong(); + totalPrimaryCount = in.readVLong(); + totalReplicaCount = in.readVLong(); + + coordinatingRejections = in.readVLong(); + coordinatingNodeLimitsBreachedRejections = in.readVLong(); + coordinatingLastSuccessfulRequestLimitsBreachedRejections = in.readVLong(); + coordinatingThroughputDegradationLimitsBreachedRejections = in.readVLong(); + + primaryRejections = in.readVLong(); + primaryNodeLimitsBreachedRejections = in.readVLong(); + primaryLastSuccessfulRequestLimitsBreachedRejections = in.readVLong(); + primaryThroughputDegradationLimitsBreachedRejections = in.readVLong(); + + replicaRejections = in.readVLong(); + replicaNodeLimitsBreachedRejections = in.readVLong(); + replicaLastSuccessfulRequestLimitsBreachedRejections = in.readVLong(); + replicaThroughputDegradationLimitsBreachedRejections = in.readVLong(); + + coordinatingTimeInMillis = in.readVLong(); + primaryTimeInMillis = in.readVLong(); + replicaTimeInMillis = in.readVLong(); + + coordinatingLastSuccessfulRequestTimestampInMillis = in.readVLong(); + primaryLastSuccessfulRequestTimestampInMillis = in.readVLong(); + replicaLastSuccessfulRequestTimestampInMillis = in.readVLong(); + + currentPrimaryAndCoordinatingLimits = in.readVLong(); + currentReplicaLimits = in.readVLong(); + } + + public IndexingPressurePerShardStats(ShardIndexingPressureTracker shardIndexingPressureTracker, + boolean shardIndexingPressureEnforced) { + + shardId = shardIndexingPressureTracker.shardId.toString(); + this.shardIndexingPressureEnforced = shardIndexingPressureEnforced; + + totalCombinedCoordinatingAndPrimaryBytes = shardIndexingPressureTracker.totalCombinedCoordinatingAndPrimaryBytes.get(); + totalCoordinatingBytes = shardIndexingPressureTracker.totalCoordinatingBytes.get(); + totalPrimaryBytes = shardIndexingPressureTracker.totalPrimaryBytes.get(); + totalReplicaBytes = shardIndexingPressureTracker.totalReplicaBytes.get(); + + currentCombinedCoordinatingAndPrimaryBytes = shardIndexingPressureTracker.currentCombinedCoordinatingAndPrimaryBytes.get(); + currentCoordinatingBytes = shardIndexingPressureTracker.currentCoordinatingBytes.get(); + currentPrimaryBytes = shardIndexingPressureTracker.currentPrimaryBytes.get(); + currentReplicaBytes = shardIndexingPressureTracker.currentReplicaBytes.get(); + + totalCoordinatingCount = shardIndexingPressureTracker.coordinatingCount.get(); + totalPrimaryCount = shardIndexingPressureTracker.primaryCount.get(); + totalReplicaCount = shardIndexingPressureTracker.replicaCount.get(); + + coordinatingRejections = shardIndexingPressureTracker.coordinatingRejections.get(); + coordinatingNodeLimitsBreachedRejections = shardIndexingPressureTracker.coordinatingNodeLimitsBreachedRejections.get(); + coordinatingLastSuccessfulRequestLimitsBreachedRejections = + shardIndexingPressureTracker.coordinatingLastSuccessfulRequestLimitsBreachedRejections.get(); + coordinatingThroughputDegradationLimitsBreachedRejections = + shardIndexingPressureTracker.coordinatingThroughputDegradationLimitsBreachedRejections.get(); + + primaryRejections = shardIndexingPressureTracker.primaryRejections.get(); + primaryNodeLimitsBreachedRejections = shardIndexingPressureTracker.primaryNodeLimitsBreachedRejections.get(); + primaryLastSuccessfulRequestLimitsBreachedRejections = shardIndexingPressureTracker + .primaryLastSuccessfulRequestLimitsBreachedRejections.get(); + primaryThroughputDegradationLimitsBreachedRejections = shardIndexingPressureTracker + .primaryThroughputDegradationLimitsBreachedRejections.get(); + + replicaRejections = shardIndexingPressureTracker.replicaRejections.get(); + replicaNodeLimitsBreachedRejections = shardIndexingPressureTracker.replicaNodeLimitsBreachedRejections.get(); + replicaLastSuccessfulRequestLimitsBreachedRejections = shardIndexingPressureTracker + .replicaLastSuccessfulRequestLimitsBreachedRejections.get(); + replicaThroughputDegradationLimitsBreachedRejections = shardIndexingPressureTracker + .replicaThroughputDegradationLimitsBreachedRejections.get(); + + coordinatingTimeInMillis = shardIndexingPressureTracker.coordinatingTimeInMillis.get(); + primaryTimeInMillis = shardIndexingPressureTracker.primaryTimeInMillis.get(); + replicaTimeInMillis = shardIndexingPressureTracker.replicaTimeInMillis.get(); + + coordinatingLastSuccessfulRequestTimestampInMillis = shardIndexingPressureTracker.lastSuccessfulCoordinatingRequestTimestamp.get(); + primaryLastSuccessfulRequestTimestampInMillis = shardIndexingPressureTracker.lastSuccessfulPrimaryRequestTimestamp.get(); + replicaLastSuccessfulRequestTimestampInMillis = shardIndexingPressureTracker.lastSuccessfulReplicaRequestTimestamp.get(); + + currentPrimaryAndCoordinatingLimits = shardIndexingPressureTracker.primaryAndCoordinatingLimits.get(); + currentReplicaLimits = shardIndexingPressureTracker.replicaLimits.get(); + + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(shardId); + out.writeBoolean(shardIndexingPressureEnforced); + + out.writeVLong(totalCombinedCoordinatingAndPrimaryBytes); + out.writeVLong(totalCoordinatingBytes); + out.writeVLong(totalPrimaryBytes); + out.writeVLong(totalReplicaBytes); + + out.writeVLong(currentCombinedCoordinatingAndPrimaryBytes); + out.writeVLong(currentCoordinatingBytes); + out.writeVLong(currentPrimaryBytes); + out.writeVLong(currentReplicaBytes); + + out.writeVLong(totalCoordinatingCount); + out.writeVLong(totalPrimaryCount); + out.writeVLong(totalReplicaCount); + + out.writeVLong(coordinatingRejections); + out.writeVLong(coordinatingNodeLimitsBreachedRejections); + out.writeVLong(coordinatingLastSuccessfulRequestLimitsBreachedRejections); + out.writeVLong(coordinatingThroughputDegradationLimitsBreachedRejections); + + out.writeVLong(primaryRejections); + out.writeVLong(primaryNodeLimitsBreachedRejections); + out.writeVLong(primaryLastSuccessfulRequestLimitsBreachedRejections); + out.writeVLong(primaryThroughputDegradationLimitsBreachedRejections); + + out.writeVLong(replicaRejections); + out.writeVLong(replicaNodeLimitsBreachedRejections); + out.writeVLong(replicaLastSuccessfulRequestLimitsBreachedRejections); + out.writeVLong(replicaThroughputDegradationLimitsBreachedRejections); + + out.writeVLong(coordinatingTimeInMillis); + out.writeVLong(primaryTimeInMillis); + out.writeVLong(replicaTimeInMillis); + + out.writeVLong(coordinatingLastSuccessfulRequestTimestampInMillis); + out.writeVLong(primaryLastSuccessfulRequestTimestampInMillis); + out.writeVLong(replicaLastSuccessfulRequestTimestampInMillis); + + out.writeVLong(currentPrimaryAndCoordinatingLimits); + out.writeVLong(currentReplicaLimits); + } + + public long getTotalCombinedCoordinatingAndPrimaryBytes() { + return totalCombinedCoordinatingAndPrimaryBytes; + } + + public long getTotalCoordinatingBytes() { + return totalCoordinatingBytes; + } + + public long getTotalPrimaryBytes() { + return totalPrimaryBytes; + } + + public long getTotalReplicaBytes() { + return totalReplicaBytes; + } + + public long getCurrentCombinedCoordinatingAndPrimaryBytes() { + return currentCombinedCoordinatingAndPrimaryBytes; + } + + public long getCurrentCoordinatingBytes() { + return currentCoordinatingBytes; + } + + public long getCurrentPrimaryBytes() { + return currentPrimaryBytes; + } + + public long getCurrentReplicaBytes() { + return currentReplicaBytes; + } + + public long getCoordinatingRejections() { + return coordinatingRejections; + } + + public long getCoordinatingNodeLimitsBreachedRejections() { + return coordinatingNodeLimitsBreachedRejections; + } + + public long getCoordinatingLastSuccessfulRequestLimitsBreachedRejections() { + return coordinatingLastSuccessfulRequestLimitsBreachedRejections; + } + + public long getCoordinatingThroughputDegradationLimitsBreachedRejections() { + return coordinatingThroughputDegradationLimitsBreachedRejections; + } + + public long getPrimaryRejections() { + return primaryRejections; + } + + public long getPrimaryNodeLimitsBreachedRejections() { + return primaryNodeLimitsBreachedRejections; + } + + public long getPrimaryLastSuccessfulRequestLimitsBreachedRejections() { + return primaryLastSuccessfulRequestLimitsBreachedRejections; + } + + public long getPrimaryThroughputDegradationLimitsBreachedRejections() { + return primaryThroughputDegradationLimitsBreachedRejections; + } + + public long getReplicaRejections() { + return replicaRejections; + } + + public long getReplicaNodeLimitsBreachedRejections() { + return replicaNodeLimitsBreachedRejections; + } + + public long getReplicaLastSuccessfulRequestLimitsBreachedRejections() { + return replicaLastSuccessfulRequestLimitsBreachedRejections; + } + + public long getReplicaThroughputDegradationLimitsBreachedRejections() { + return replicaThroughputDegradationLimitsBreachedRejections; + } + + public long getCurrentPrimaryAndCoordinatingLimits() { + return currentPrimaryAndCoordinatingLimits; + } + + public long getCurrentReplicaLimits() { + return currentReplicaLimits; + } + + private static final String COORDINATING = "coordinating"; + private static final String COORDINATING_IN_BYTES = "coordinating_in_bytes"; + private static final String COORDINATING_COUNT = "coordinating_count"; + private static final String PRIMARY = "primary"; + private static final String PRIMARY_IN_BYTES = "primary_in_bytes"; + private static final String PRIMARY_COUNT = "primary_count"; + private static final String REPLICA = "replica"; + private static final String REPLICA_IN_BYTES = "replica_in_bytes"; + private static final String REPLICA_COUNT = "replica_count"; + private static final String COORDINATING_REJECTIONS = "coordinating_rejections"; + private static final String PRIMARY_REJECTIONS = "primary_rejections"; + private static final String REPLICA_REJECTIONS = "replica_rejections"; + private static final String BREAKUP_NODE_LIMITS = "node_limits"; + private static final String BREAKUP_NO_SUCCESSFUL_REQUEST_LIMITS = "no_successful_request_limits"; + private static final String BREAKUP_THROUGHPUT_DEGRADATION_LIMIT = "throughput_degradation_limits"; + private static final String COORDINATING_TIME_IN_MILLIS = "coordinating_time_in_millis"; + private static final String PRIMARY_TIME_IN_MILLIS = "primary_time_in_millis"; + private static final String REPLICA_TIME_IN_MILLIS = "replica_time_in_millis"; + private static final String COORDINATING_LAST_SUCCESSFUL_REQUEST_TIMESTAMP_IN_MILLIS = + "coordinating_last_successful_request_timestamp_in_millis"; + private static final String PRIMARY_LAST_SUCCESSFUL_REQUEST_TIMESTAMP_IN_MILLIS = + "primary_last_successful_request_timestamp_in_millis"; + private static final String REPLICA_LAST_SUCCESSFUL_REQUEST_TIMESTAMP_IN_MILLIS = "replica_last_successful_request_timestamp_in_millis"; + private static final String CURRENT_COORDINATING_AND_PRIMARY_LIMITS_IN_BYTES = "current_coordinating_and_primary_limits_in_bytes"; + private static final String CURRENT_REPLICA_LIMITS_IN_BYTES = "current_replica_limits_in_bytes"; + private static final String CURRENT_COORDINATING_AND_PRIMARY_IN_BYTES = "current_coordinating_and_primary_bytes"; + private static final String CURRENT_REPLICA_IN_BYTES = "current_replica_bytes"; + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(shardId); + + builder.startObject("memory"); + builder.startObject("current"); + builder.humanReadableField(COORDINATING_IN_BYTES, COORDINATING, new ByteSizeValue(currentCoordinatingBytes)); + builder.humanReadableField(PRIMARY_IN_BYTES, PRIMARY, new ByteSizeValue(currentPrimaryBytes)); + builder.humanReadableField(REPLICA_IN_BYTES, REPLICA, new ByteSizeValue(currentReplicaBytes)); + builder.endObject(); + builder.startObject("total"); + builder.humanReadableField(COORDINATING_IN_BYTES, COORDINATING, new ByteSizeValue(totalCoordinatingBytes)); + builder.humanReadableField(PRIMARY_IN_BYTES, PRIMARY, new ByteSizeValue(totalPrimaryBytes)); + builder.humanReadableField(REPLICA_IN_BYTES, REPLICA, new ByteSizeValue(totalReplicaBytes)); + builder.endObject(); + builder.endObject(); + + builder.startObject("rejection"); + builder.startObject("coordinating"); + builder.field(COORDINATING_REJECTIONS, coordinatingRejections); + if (shardIndexingPressureEnforced) { + builder.startObject("breakup"); + } else { + builder.startObject("breakup_shadow_mode"); + } + builder.field(BREAKUP_NODE_LIMITS, coordinatingNodeLimitsBreachedRejections); + builder.field(BREAKUP_NO_SUCCESSFUL_REQUEST_LIMITS, coordinatingLastSuccessfulRequestLimitsBreachedRejections); + builder.field(BREAKUP_THROUGHPUT_DEGRADATION_LIMIT, coordinatingThroughputDegradationLimitsBreachedRejections); + builder.endObject(); + builder.endObject(); + builder.startObject("primary"); + builder.field(PRIMARY_REJECTIONS, primaryRejections); + if (shardIndexingPressureEnforced) { + builder.startObject("breakup"); + } else { + builder.startObject("breakup_shadow_mode"); + } + builder.field(BREAKUP_NODE_LIMITS, primaryNodeLimitsBreachedRejections); + builder.field(BREAKUP_NO_SUCCESSFUL_REQUEST_LIMITS, primaryLastSuccessfulRequestLimitsBreachedRejections); + builder.field(BREAKUP_THROUGHPUT_DEGRADATION_LIMIT, primaryThroughputDegradationLimitsBreachedRejections); + builder.endObject(); + builder.endObject(); + builder.startObject("replica"); + builder.field(REPLICA_REJECTIONS, replicaRejections); + if (shardIndexingPressureEnforced) { + builder.startObject("breakup"); + } else { + builder.startObject("breakup_shadow_mode"); + } + builder.field(BREAKUP_NODE_LIMITS, replicaNodeLimitsBreachedRejections); + builder.field(BREAKUP_NO_SUCCESSFUL_REQUEST_LIMITS, replicaLastSuccessfulRequestLimitsBreachedRejections); + builder.field(BREAKUP_THROUGHPUT_DEGRADATION_LIMIT, replicaThroughputDegradationLimitsBreachedRejections); + builder.endObject(); + builder.endObject(); + builder.endObject(); + + builder.startObject("last_successful_timestamp"); + builder.field(COORDINATING_LAST_SUCCESSFUL_REQUEST_TIMESTAMP_IN_MILLIS, coordinatingLastSuccessfulRequestTimestampInMillis); + builder.field(PRIMARY_LAST_SUCCESSFUL_REQUEST_TIMESTAMP_IN_MILLIS, primaryLastSuccessfulRequestTimestampInMillis); + builder.field(REPLICA_LAST_SUCCESSFUL_REQUEST_TIMESTAMP_IN_MILLIS, replicaLastSuccessfulRequestTimestampInMillis); + builder.endObject(); + + builder.startObject("indexing"); + builder.field(COORDINATING_TIME_IN_MILLIS, coordinatingTimeInMillis); + builder.field(COORDINATING_COUNT, totalCoordinatingCount); + builder.field(PRIMARY_TIME_IN_MILLIS, primaryTimeInMillis); + builder.field(PRIMARY_COUNT, totalPrimaryCount); + builder.field(REPLICA_TIME_IN_MILLIS, replicaTimeInMillis); + builder.field(REPLICA_COUNT, totalReplicaCount); + builder.endObject(); + + builder.startObject("memory_allocation"); + builder.startObject("current"); + builder.field(CURRENT_COORDINATING_AND_PRIMARY_IN_BYTES, currentCombinedCoordinatingAndPrimaryBytes); + builder.field(CURRENT_REPLICA_IN_BYTES, currentReplicaBytes); + builder.endObject(); + builder.startObject("limit"); + builder.field(CURRENT_COORDINATING_AND_PRIMARY_LIMITS_IN_BYTES, currentPrimaryAndCoordinatingLimits); + builder.field(CURRENT_REPLICA_LIMITS_IN_BYTES, currentReplicaLimits); + builder.endObject(); + builder.endObject(); + + return builder.endObject(); + } +} diff --git a/server/src/main/java/org/opensearch/index/stats/ShardIndexingPressureStats.java b/server/src/main/java/org/opensearch/index/stats/ShardIndexingPressureStats.java new file mode 100644 index 0000000000000..73e7301ba7345 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/stats/ShardIndexingPressureStats.java @@ -0,0 +1,104 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index.stats; + +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.common.xcontent.ToXContent; +import org.opensearch.common.xcontent.ToXContentFragment; +import org.opensearch.common.xcontent.XContentBuilder; +import org.opensearch.index.shard.ShardId; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class ShardIndexingPressureStats implements Writeable, ToXContentFragment { + + private final Map shardIndexingPressureStore; + private final long totalNodeLimitsBreachedRejections; + private final long totalLastSuccessfulRequestLimitsBreachedRejections; + private final long totalThroughputDegradationLimitsBreachedRejections; + private final boolean shardIndexingPressureEnabled; + private final boolean shardIndexingPressureEnforced; + + public ShardIndexingPressureStats(StreamInput in) throws IOException { + int shardEntries = in.readInt(); + shardIndexingPressureStore = new HashMap<>(); + for (int i = 0; i < shardEntries; i++) { + Long hashCode = in.readLong(); + IndexingPressurePerShardStats shardStats = new IndexingPressurePerShardStats(in); + shardIndexingPressureStore.put(hashCode, shardStats); + } + totalNodeLimitsBreachedRejections = in.readVLong(); + totalLastSuccessfulRequestLimitsBreachedRejections = in.readVLong(); + totalThroughputDegradationLimitsBreachedRejections = in.readVLong(); + shardIndexingPressureEnabled = in.readBoolean(); + shardIndexingPressureEnforced = in.readBoolean(); + } + + public ShardIndexingPressureStats(Map shardIndexingPressureStore, + long totalNodeLimitsBreachedRejections, + long totalLastSuccessfulRequestLimitsBreachedRejections, + long totalThroughputDegradationLimitsBreachedRejections, + boolean shardIndexingPressureEnabled, + boolean shardIndexingPressureEnforced) { + this.shardIndexingPressureStore = shardIndexingPressureStore; + this.totalNodeLimitsBreachedRejections = totalNodeLimitsBreachedRejections; + this.totalLastSuccessfulRequestLimitsBreachedRejections = totalLastSuccessfulRequestLimitsBreachedRejections; + this.totalThroughputDegradationLimitsBreachedRejections = totalThroughputDegradationLimitsBreachedRejections; + this.shardIndexingPressureEnabled = shardIndexingPressureEnabled; + this.shardIndexingPressureEnforced = shardIndexingPressureEnforced; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeInt(shardIndexingPressureStore.size()); + for (Map.Entry entry : shardIndexingPressureStore.entrySet()) { + out.writeLong(entry.getKey()); + entry.getValue().writeTo(out); + } + out.writeVLong(totalNodeLimitsBreachedRejections); + out.writeVLong(totalLastSuccessfulRequestLimitsBreachedRejections); + out.writeVLong(totalThroughputDegradationLimitsBreachedRejections); + out.writeBoolean(shardIndexingPressureEnabled); + out.writeBoolean(shardIndexingPressureEnforced); + } + + public IndexingPressurePerShardStats getIndexingPressureShardStats(ShardId shardId) { + IndexingPressurePerShardStats value = shardIndexingPressureStore.get((long)shardId.hashCode()); + return value; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject("shard_indexing_pressure"); + builder.startObject("stats"); + for (Map.Entry entry : shardIndexingPressureStore.entrySet()) { + entry.getValue().toXContent(builder, params); + } + builder.endObject(); + if (shardIndexingPressureEnforced) { + builder.startObject("total_rejections_breakup"); + } else { + builder.startObject("total_rejections_breakup_shadow_mode"); + } + builder.field("node_limits", totalNodeLimitsBreachedRejections); + builder.field("no_successful_request_limits", totalLastSuccessfulRequestLimitsBreachedRejections); + builder.field("throughput_degradation_limits", totalThroughputDegradationLimitsBreachedRejections); + builder.endObject(); + builder.field("enabled", shardIndexingPressureEnabled); + builder.field("enforced", shardIndexingPressureEnforced); + return builder.endObject(); + } + + public void addAll(ShardIndexingPressureStats shardIndexingPressureStats) { + if (this.shardIndexingPressureStore != null) { + this.shardIndexingPressureStore.putAll(shardIndexingPressureStats.shardIndexingPressureStore); + } + } +} diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index da9041d38d66c..54e903bb3efb7 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -205,6 +205,7 @@ import java.util.stream.Stream; import static java.util.stream.Collectors.toList; +import static org.opensearch.index.ShardIndexingPressure.SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY; /** * A node represent a node within a cluster ({@code cluster.name}). The {@link #client()} can be used @@ -304,6 +305,9 @@ protected Node(final Environment initialEnvironment, Settings tmpSettings = Settings.builder().put(initialEnvironment.settings()) .put(Client.CLIENT_TYPE_SETTING_S.getKey(), CLIENT_TYPE).build(); + // Enabling shard indexing backpressure + tmpSettings = addShardIndexingBackPressureAttributeSettings(tmpSettings); + final JvmInfo jvmInfo = JvmInfo.jvmInfo(); logger.info( "version[{}], pid[{}], build[{}/{}/{}], OS[{}/{}/{}], JVM[{}/{}/{}/{}]", @@ -586,7 +590,8 @@ protected Node(final Environment initialEnvironment, final SearchTransportService searchTransportService = new SearchTransportService(transportService, SearchExecutionStatsCollector.makeWrapper(responseCollectorService)); final HttpServerTransport httpServerTransport = newHttpTransport(networkModule); - final IndexingPressure indexingLimits = new IndexingPressure(settings); + final IndexingPressure indexingLimits = new IndexingPressure(settings, clusterService); + clusterService.setIndexingPressure(indexingLimits); final RecoverySettings recoverySettings = new RecoverySettings(settings, settingsModule.getClusterSettings()); RepositoriesModule repositoriesModule = new RepositoriesModule(this.environment, @@ -732,6 +737,14 @@ protected Node(final Environment initialEnvironment, } } + private static Settings addShardIndexingBackPressureAttributeSettings(Settings settings) { + // Shard Indexing BackPressure is enabled from AES-7.9 onwards. + String ShardIndexingBackPressureEnabledValue = "true"; + return Settings.builder().put(settings) + .put(NODE_ATTRIBUTES.getKey() + SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY, ShardIndexingBackPressureEnabledValue) + .build(); + } + protected TransportService newTransportService(Settings settings, Transport transport, ThreadPool threadPool, TransportInterceptor interceptor, Function localNodeFactory, diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java index 40362eeb4060e..7e08dbdfdf090 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIndicesThatCannotBeCreatedTests.java @@ -31,6 +31,7 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AtomicArray; @@ -125,7 +126,9 @@ private void indicesThatCannotBeCreatedTestCase(Set expected, final ExecutorService direct = OpenSearchExecutors.newDirectExecutorService(); when(threadPool.executor(anyString())).thenReturn(direct); TransportBulkAction action = new TransportBulkAction(threadPool, mock(TransportService.class), clusterService, - null, null, null, mock(ActionFilters.class), null, null, new IndexingPressure(Settings.EMPTY), new SystemIndices(emptyMap())) { + null, null, null, mock(ActionFilters.class), null, null, new IndexingPressure(Settings.EMPTY, + new ClusterService(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + null)), new SystemIndices(emptyMap())) { @Override void executeBulk(Task task, BulkRequest bulkRequest, long startTimeNanos, ActionListener listener, AtomicArray responses, Map indicesThatCannotBeCreated) { diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java index 816a607d01d79..04b3d1cb82813 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java @@ -150,7 +150,8 @@ null, null, new ActionFilters(Collections.emptySet()), null, SETTINGS, new ClusterSettings(SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), new SystemIndices(emptyMap()) - ), new IndexingPressure(SETTINGS), new SystemIndices(emptyMap()) + ), new IndexingPressure(SETTINGS, new ClusterService(SETTINGS, new ClusterSettings(SETTINGS, + ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null)), new SystemIndices(emptyMap()) ); } @Override diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java index 65431e60c9d2f..adce6f2c0ad6d 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java @@ -92,7 +92,8 @@ class TestTransportBulkAction extends TransportBulkAction { super(TransportBulkActionTests.this.threadPool, transportService, clusterService, null, null, null, new ActionFilters(Collections.emptySet()), new Resolver(), new AutoCreateIndex(Settings.EMPTY, clusterService.getClusterSettings(), new Resolver(), new SystemIndices(emptyMap())), - new IndexingPressure(Settings.EMPTY), new SystemIndices(emptyMap())); + new IndexingPressure(Settings.EMPTY, clusterService), + new SystemIndices(emptyMap())); } @Override diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTookTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTookTests.java index a86032458f579..3c8676d29dc70 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTookTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTookTests.java @@ -253,7 +253,7 @@ static class TestTransportBulkAction extends TransportBulkAction { actionFilters, indexNameExpressionResolver, autoCreateIndex, - new IndexingPressure(Settings.EMPTY), + new IndexingPressure(Settings.EMPTY, clusterService), new SystemIndices(emptyMap()), relativeTimeProvider); } diff --git a/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java b/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java index 3670160f9c3c9..34599024df87c 100644 --- a/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java +++ b/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java @@ -149,7 +149,8 @@ public void testResyncDoesNotBlockOnPrimaryAction() throws Exception { final TransportResyncReplicationAction action = new TransportResyncReplicationAction(Settings.EMPTY, transportService, clusterService, indexServices, threadPool, shardStateAction, new ActionFilters(new HashSet<>()), - new IndexingPressure(Settings.EMPTY), new SystemIndices(emptyMap())); + new IndexingPressure(Settings.EMPTY, clusterService), + new SystemIndices(emptyMap())); assertThat(action.globalBlockLevel(), nullValue()); assertThat(action.indexBlockLevel(), nullValue()); diff --git a/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionForIndexingPressureTests.java b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionForIndexingPressureTests.java new file mode 100644 index 0000000000000..655d7bfa01a21 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionForIndexingPressureTests.java @@ -0,0 +1,516 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.action.support.replication; + +import org.opensearch.action.ActionListener; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.action.support.WriteResponse; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.action.shard.ShardStateAction; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Nullable; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.Index; +import org.opensearch.index.IndexService; +import org.opensearch.index.IndexingPressure; +import org.opensearch.index.ShardIndexingPressure; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardState; +import org.opensearch.index.shard.ReplicationGroup; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.shard.ShardNotFoundException; +import org.opensearch.index.shard.ShardNotInPrimaryModeException; +import org.opensearch.index.translog.Translog; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.SystemIndices; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.transport.CapturingTransport; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.Transport; +import org.opensearch.transport.TransportChannel; +import org.opensearch.transport.TransportResponse; +import org.opensearch.transport.TransportService; +import org.hamcrest.Matcher; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Locale; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static java.util.Collections.emptyMap; +import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; +import static org.opensearch.test.ClusterServiceUtils.setState; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.never; + +public class TransportWriteActionForIndexingPressureTests extends OpenSearchTestCase { + private static ThreadPool threadPool; + + private ClusterService clusterService; + private TransportService transportService; + private CapturingTransport transport; + private ShardStateAction shardStateAction; + private Translog.Location location; + private IndexingPressure mockIndexingPressure; + private ShardIndexingPressure mockShardIndexingPressure; + private Releasable releasable; + + public static final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + + @BeforeClass + public static void beforeClass() { + threadPool = new TestThreadPool("ShardReplicationTests"); + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + transport = new CapturingTransport(); + clusterService = createClusterService(threadPool); + transportService = transport.createTransportService(clusterService.getSettings(), threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); + transportService.start(); + transportService.acceptIncomingRequests(); + shardStateAction = new ShardStateAction(clusterService, transportService, null, null, threadPool); + mockIndexingPressure = mock(IndexingPressure.class); + mockShardIndexingPressure = mock(ShardIndexingPressure.class); + releasable = mock(Releasable.class); + location = mock(Translog.Location.class); + } + + @Override + @After + public void tearDown() throws Exception { + super.tearDown(); + clusterService.close(); + } + + @AfterClass + public static void afterClass() { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + threadPool = null; + } + + public void testIndexingPressureOperationStartedForReplicaNode() { + final ShardId shardId = new ShardId("test", "_na_", 0); + final ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED); + setState(clusterService, state); + final ShardRouting replicaRouting = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0); + final ReplicationTask task = maybeTask(); + + when(mockIndexingPressure.getShardIndexingPressure()).thenReturn(mockShardIndexingPressure); + when(mockIndexingPressure.markReplicaOperationStarted(anyLong(), anyBoolean())).thenReturn(releasable); + + when(mockIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(false); + when(mockShardIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(false); + + TestAction action = new TestAction(Settings.EMPTY, "internal:testAction", transportService, clusterService, + shardStateAction, threadPool); + + action.handleReplicaRequest( + new TransportReplicationAction.ConcreteReplicaRequest<>( + new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(), + randomNonNegativeLong(), randomNonNegativeLong()), + createTransportChannel(new PlainActionFuture<>()), task); + + assertPhase(task, "finished"); + verify(mockIndexingPressure, times(1)).markReplicaOperationStarted(anyLong(), anyBoolean()); + verify(mockShardIndexingPressure, never()).markReplicaOperationStarted(anyObject(), anyLong(), anyBoolean()); + verify(releasable, times(1)).close(); + } + + public void testIndexingPressureOperationStartedForReplicaShard() { + final ShardId shardId = new ShardId("test", "_na_", 0); + final ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED); + setState(clusterService, state); + final ShardRouting replicaRouting = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0); + final ReplicationTask task = maybeTask(); + + when(mockIndexingPressure.getShardIndexingPressure()).thenReturn(mockShardIndexingPressure); + when(mockShardIndexingPressure.markReplicaOperationStarted(any(), anyLong(), anyBoolean())).thenReturn(releasable); + + when(mockIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(true); + when(mockShardIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(true); + + TestAction action = new TestAction(Settings.EMPTY, "internal:testAction", transportService, clusterService, + shardStateAction, threadPool); + + action.handleReplicaRequest( + new TransportReplicationAction.ConcreteReplicaRequest<>( + new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(), + randomNonNegativeLong(), randomNonNegativeLong()), + createTransportChannel(new PlainActionFuture<>()), task); + + assertPhase(task, "finished"); + verify(mockIndexingPressure, never()).markReplicaOperationStarted(anyLong(), anyBoolean()); + verify(mockShardIndexingPressure, times(1)).markReplicaOperationStarted(any(), anyLong(), anyBoolean()); + verify(releasable, times(1)).close(); + } + + public void testIndexingPressureOperationStartedForPrimaryNode() { + final ShardId shardId = new ShardId("test", "_na_", 0); + final ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED); + setState(clusterService, state); + final ShardRouting replicaRouting = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0); + final ReplicationTask task = maybeTask(); + + when(mockIndexingPressure.getShardIndexingPressure()).thenReturn(mockShardIndexingPressure); + when(mockIndexingPressure.markPrimaryOperationStarted(anyLong(), anyBoolean())).thenReturn(releasable); + + when(mockIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(false); + when(mockShardIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(false); + + TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService, + shardStateAction, threadPool); + + action.handlePrimaryRequest( + new TransportReplicationAction.ConcreteReplicaRequest<>( + new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(), + randomNonNegativeLong(), randomNonNegativeLong()), + createTransportChannel(new PlainActionFuture<>()), task); + assertPhase(task, "finished"); + verify(mockIndexingPressure, times(1)).markPrimaryOperationStarted(anyLong(), anyBoolean()); + verify(mockShardIndexingPressure, never()).markPrimaryOperationStarted(any(), anyLong(), anyBoolean()); + verify(releasable, times(1)).close(); + } + + public void testIndexingPressureOperationStartedForPrimaryShard() { + final ShardId shardId = new ShardId("test", "_na_", 0); + final ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED); + setState(clusterService, state); + final ShardRouting replicaRouting = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0); + final ReplicationTask task = maybeTask(); + + when(mockIndexingPressure.getShardIndexingPressure()).thenReturn(mockShardIndexingPressure); + when(mockShardIndexingPressure.markPrimaryOperationStarted(any(), anyLong(), anyBoolean())).thenReturn(releasable); + + when(mockIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(true); + when(mockShardIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(true); + + TestAction action = new TestAction(Settings.EMPTY, "internal:testActionWithExceptions", transportService, clusterService, + shardStateAction, threadPool); + + action.handlePrimaryRequest( + new TransportReplicationAction.ConcreteReplicaRequest<>( + new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(), + randomNonNegativeLong(), randomNonNegativeLong()), + createTransportChannel(new PlainActionFuture<>()), task); + assertPhase(task, "finished"); + verify(mockIndexingPressure, never()).markPrimaryOperationStarted(anyLong(), anyBoolean()); + verify(mockShardIndexingPressure, times(1)).markPrimaryOperationStarted(any(), anyLong(), anyBoolean()); + verify(releasable, times(1)).close(); + } + + public void testIndexingPressureOperationStartedForLocalPrimaryNode() { + final ShardId shardId = new ShardId("test", "_na_", 0); + final ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED); + setState(clusterService, state); + final ShardRouting replicaRouting = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0); + final ReplicationTask task = maybeTask(); + + when(mockIndexingPressure.getShardIndexingPressure()).thenReturn(mockShardIndexingPressure); + when(mockIndexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(anyLong())).thenReturn(releasable); + + when(mockIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(false); + when(mockShardIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(false); + + TestAction action = new TestAction(Settings.EMPTY, "internal:testAction", transportService, clusterService, + shardStateAction, threadPool); + + action.handlePrimaryRequest( + new TransportReplicationAction.ConcreteShardRequest<>( + new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(), + true, true), + createTransportChannel(new PlainActionFuture<>()), task); + assertPhase(task, "finished"); + verify(mockIndexingPressure, times(1)).markPrimaryOperationLocalToCoordinatingNodeStarted(anyLong()); + verify(mockShardIndexingPressure, never()).markPrimaryOperationLocalToCoordinatingNodeStarted(any(), anyLong()); + verify(releasable, times(1)).close(); + } + + public void testIndexingPressureOperationStartedForLocalPrimaryShard() { + final ShardId shardId = new ShardId("test", "_na_", 0); + final ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED); + setState(clusterService, state); + final ShardRouting replicaRouting = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0); + final ReplicationTask task = maybeTask(); + + when(mockIndexingPressure.getShardIndexingPressure()).thenReturn(mockShardIndexingPressure); + when(mockShardIndexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(any(), anyLong())).thenReturn(releasable); + + when(mockIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(true); + when(mockShardIndexingPressure.isShardIndexingPressureEnabled()).thenReturn(true); + + TestAction action = new TestAction(Settings.EMPTY, "internal:testAction", transportService, clusterService, + shardStateAction, threadPool); + + action.handlePrimaryRequest( + new TransportReplicationAction.ConcreteShardRequest<>( + new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(), + true, true), + createTransportChannel(new PlainActionFuture<>()), task); + assertPhase(task, "finished"); + verify(mockIndexingPressure, never()).markPrimaryOperationLocalToCoordinatingNodeStarted(anyLong()); + verify(mockShardIndexingPressure, times(1)).markPrimaryOperationLocalToCoordinatingNodeStarted(any(), anyLong()); + verify(releasable, times(1)).close(); + } + + private final AtomicInteger count = new AtomicInteger(0); + + private final AtomicBoolean isRelocated = new AtomicBoolean(false); + + private final AtomicBoolean isPrimaryMode = new AtomicBoolean(true); + + /** + * Sometimes build a ReplicationTask for tracking the phase of the + * TransportReplicationAction. Since TransportReplicationAction has to work + * if the task as null just as well as if it is supplied this returns null + * half the time. + */ + ReplicationTask maybeTask() { + return random().nextBoolean() ? new ReplicationTask(0, null, null, null, null, null) : null; + } + + /** + * If the task is non-null this asserts that the phrase matches. + */ + void assertPhase(@Nullable ReplicationTask task, String phase) { + assertPhase(task, equalTo(phase)); + } + + private void assertPhase(@Nullable ReplicationTask task, Matcher phaseMatcher) { + if (task != null) { + assertThat(task.getPhase(), phaseMatcher); + } + } + + private class TestAction extends TransportWriteAction { + + private final boolean withDocumentFailureOnPrimary; + private final boolean withDocumentFailureOnReplica; + + protected TestAction() { + this(false, false); + } + + protected TestAction(boolean withDocumentFailureOnPrimary, boolean withDocumentFailureOnReplica) { + super(Settings.EMPTY, "internal:test", + new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, + x -> null, null, Collections.emptySet()), TransportWriteActionForIndexingPressureTests.this.clusterService, null, null, + null, new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ignore -> ThreadPool.Names.SAME, false, + mockIndexingPressure, new SystemIndices(emptyMap())); + this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary; + this.withDocumentFailureOnReplica = withDocumentFailureOnReplica; + } + + protected TestAction(Settings settings, String actionName, TransportService transportService, + ClusterService clusterService, ShardStateAction shardStateAction, ThreadPool threadPool) { + super(settings, actionName, transportService, clusterService, + mockIndicesService(clusterService), threadPool, shardStateAction, + new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ignore -> ThreadPool.Names.SAME, false, + mockIndexingPressure, new SystemIndices(emptyMap())); + this.withDocumentFailureOnPrimary = false; + this.withDocumentFailureOnReplica = false; + } + + + @Override + protected TestResponse newResponseInstance(StreamInput in) throws IOException { + return new TestResponse(); + } + + @Override + protected void dispatchedShardOperationOnPrimary( + TestRequest request, IndexShard primary, ActionListener> listener) { + ActionListener.completeWith(listener, () -> { + if (withDocumentFailureOnPrimary) { + return new WritePrimaryResult<>(request, null, null, new RuntimeException("simulated"), primary, logger); + } else { + return new WritePrimaryResult<>(request, new TestResponse(), location, null, primary, logger); + } + }); + } + + @Override + protected void dispatchedShardOperationOnReplica(TestRequest request, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + final WriteReplicaResult replicaResult; + if (withDocumentFailureOnReplica) { + replicaResult = new WriteReplicaResult<>(request, null, new RuntimeException("simulated"), replica, logger); + } else { + replicaResult = new WriteReplicaResult<>(request, location, null, replica, logger); + } + return replicaResult; + }); + } + } + + private static class TestRequest extends ReplicatedWriteRequest { + TestRequest(StreamInput in) throws IOException { + super(in); + } + + TestRequest() { + super(new ShardId("test", "_na_", 0)); + } + + @Override + public String toString() { + return "TestRequest{}"; + } + } + + private static class TestResponse extends ReplicationResponse implements WriteResponse { + boolean forcedRefresh; + + @Override + public void setForcedRefresh(boolean forcedRefresh) { + this.forcedRefresh = forcedRefresh; + } + } + + private IndicesService mockIndicesService(ClusterService clusterService) { + final IndicesService indicesService = mock(IndicesService.class); + when(indicesService.indexServiceSafe(any(Index.class))).then(invocation -> { + Index index = (Index)invocation.getArguments()[0]; + final ClusterState state = clusterService.state(); + final IndexMetadata indexSafe = state.metadata().getIndexSafe(index); + return mockIndexService(indexSafe, clusterService); + }); + when(indicesService.indexService(any(Index.class))).then(invocation -> { + Index index = (Index) invocation.getArguments()[0]; + final ClusterState state = clusterService.state(); + if (state.metadata().hasIndex(index.getName())) { + return mockIndexService(clusterService.state().metadata().getIndexSafe(index), clusterService); + } else { + return null; + } + }); + return indicesService; + } + + private IndexService mockIndexService(final IndexMetadata indexMetaData, ClusterService clusterService) { + final IndexService indexService = mock(IndexService.class); + when(indexService.getShard(anyInt())).then(invocation -> { + int shard = (Integer) invocation.getArguments()[0]; + final ShardId shardId = new ShardId(indexMetaData.getIndex(), shard); + if (shard > indexMetaData.getNumberOfShards()) { + throw new ShardNotFoundException(shardId); + } + return mockIndexShard(shardId, clusterService); + }); + return indexService; + } + + @SuppressWarnings("unchecked") + private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService) { + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.shardId()).thenReturn(shardId); + when(indexShard.state()).thenReturn(IndexShardState.STARTED); + doAnswer(invocation -> { + ActionListener callback = (ActionListener) invocation.getArguments()[0]; + if (isPrimaryMode.get()) { + count.incrementAndGet(); + callback.onResponse(count::decrementAndGet); + + } else { + callback.onFailure(new ShardNotInPrimaryModeException(shardId, IndexShardState.STARTED)); + } + return null; + }).when(indexShard).acquirePrimaryOperationPermit(any(ActionListener.class), anyString(), anyObject()); + doAnswer(invocation -> { + long term = (Long)invocation.getArguments()[0]; + ActionListener callback = (ActionListener) invocation.getArguments()[3]; + final long primaryTerm = indexShard.getPendingPrimaryTerm(); + if (term < primaryTerm) { + throw new IllegalArgumentException(String.format(Locale.ROOT, "%s operation term [%d] is too old (current [%d])", + shardId, term, primaryTerm)); + } + count.incrementAndGet(); + callback.onResponse(count::decrementAndGet); + return null; + }).when(indexShard) + .acquireReplicaOperationPermit(anyLong(), anyLong(), anyLong(), any(ActionListener.class), anyString(), anyObject()); + when(indexShard.getActiveOperationsCount()).thenAnswer(i -> count.get()); + + when(indexShard.routingEntry()).thenAnswer(invocationOnMock -> { + final ClusterState state = clusterService.state(); + final RoutingNode node = state.getRoutingNodes().node(state.nodes().getLocalNodeId()); + final ShardRouting routing = node.getByShardId(shardId); + if (routing == null) { + throw new ShardNotFoundException(shardId, "shard is no longer assigned to current node"); + } + return routing; + }); + when(indexShard.isRelocatedPrimary()).thenAnswer(invocationOnMock -> isRelocated.get()); + doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class)); + when(indexShard.getPendingPrimaryTerm()).thenAnswer(i -> + clusterService.state().metadata().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id())); + + ReplicationGroup replicationGroup = mock(ReplicationGroup.class); + when(indexShard.getReplicationGroup()).thenReturn(replicationGroup); + return indexShard; + } + + /** + * Transport channel that is needed for testing. + */ + public TransportChannel createTransportChannel(final PlainActionFuture listener) { + return new TransportChannel() { + + @Override + public String getProfileName() { + return ""; + } + + @Override + public void sendResponse(TransportResponse response) { + listener.onResponse(((TestResponse) response)); + } + + @Override + public void sendResponse(Exception exception) { + listener.onFailure(exception); + } + + @Override + public String getChannelType() { + return "replica_test"; + } + }; + } + +} diff --git a/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java index 0f05849bb4392..02923bf4a15ad 100644 --- a/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java @@ -369,7 +369,8 @@ protected TestAction(boolean withDocumentFailureOnPrimary, boolean withDocumentF new TransportService(Settings.EMPTY, mock(Transport.class), null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()), TransportWriteActionTests.this.clusterService, null, null, null, new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ignore -> ThreadPool.Names.SAME, false, - new IndexingPressure(Settings.EMPTY), new SystemIndices(emptyMap())); + new IndexingPressure(Settings.EMPTY, TransportWriteActionTests.this.clusterService), + new SystemIndices(emptyMap())); this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary; this.withDocumentFailureOnReplica = withDocumentFailureOnReplica; } @@ -379,7 +380,7 @@ protected TestAction(Settings settings, String actionName, TransportService tran super(settings, actionName, transportService, clusterService, mockIndicesService(clusterService), threadPool, shardStateAction, new ActionFilters(new HashSet<>()), TestRequest::new, TestRequest::new, ignore -> ThreadPool.Names.SAME, false, - new IndexingPressure(settings), new SystemIndices(emptyMap())); + new IndexingPressure(settings, clusterService), new SystemIndices(emptyMap())); this.withDocumentFailureOnPrimary = false; this.withDocumentFailureOnReplica = false; } diff --git a/server/src/test/java/org/opensearch/index/IndexingPressureTests.java b/server/src/test/java/org/opensearch/index/IndexingPressureTests.java index 583088f590851..dbbe6b87092c0 100644 --- a/server/src/test/java/org/opensearch/index/IndexingPressureTests.java +++ b/server/src/test/java/org/opensearch/index/IndexingPressureTests.java @@ -19,7 +19,9 @@ package org.opensearch.index; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.lease.Releasable; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException; import org.opensearch.index.stats.IndexingPressureStats; @@ -27,10 +29,14 @@ public class IndexingPressureTests extends OpenSearchTestCase { - private final Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB").build(); + private final Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false).build(); + + final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + final ClusterService clusterService = new ClusterService(settings, clusterSettings, null); public void testMemoryBytesMarkedAndReleased() { - IndexingPressure indexingPressure = new IndexingPressure(settings); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); try (Releasable coordinating = indexingPressure.markCoordinatingOperationStarted(10, false); Releasable coordinating2 = indexingPressure.markCoordinatingOperationStarted(50, false); Releasable primary = indexingPressure.markPrimaryOperationStarted(15, true); @@ -55,7 +61,7 @@ public void testMemoryBytesMarkedAndReleased() { } public void testAvoidDoubleAccounting() { - IndexingPressure indexingPressure = new IndexingPressure(settings); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); try (Releasable coordinating = indexingPressure.markCoordinatingOperationStarted(10, false); Releasable primary = indexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(15)) { IndexingPressureStats stats = indexingPressure.stats(); @@ -73,7 +79,7 @@ public void testAvoidDoubleAccounting() { } public void testCoordinatingPrimaryRejections() { - IndexingPressure indexingPressure = new IndexingPressure(settings); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); try (Releasable coordinating = indexingPressure.markCoordinatingOperationStarted(1024 * 3, false); Releasable primary = indexingPressure.markPrimaryOperationStarted(1024 * 3, false); Releasable replica = indexingPressure.markReplicaOperationStarted(1024 * 3, false)) { @@ -110,7 +116,7 @@ public void testCoordinatingPrimaryRejections() { } public void testReplicaRejections() { - IndexingPressure indexingPressure = new IndexingPressure(settings); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); try (Releasable coordinating = indexingPressure.markCoordinatingOperationStarted(1024 * 3, false); Releasable primary = indexingPressure.markPrimaryOperationStarted(1024 * 3, false); Releasable replica = indexingPressure.markReplicaOperationStarted(1024 * 3, false)) { @@ -135,7 +141,7 @@ public void testReplicaRejections() { } public void testForceExecutionOnCoordinating() { - IndexingPressure indexingPressure = new IndexingPressure(settings); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); expectThrows(OpenSearchRejectedExecutionException.class, () -> indexingPressure.markCoordinatingOperationStarted(1024 * 11, false)); try (Releasable ignore = indexingPressure.markCoordinatingOperationStarted(1024 * 11, true)) { assertEquals(1024 * 11, indexingPressure.stats().getCurrentCoordinatingBytes()); diff --git a/server/src/test/java/org/opensearch/index/ShardIndexingPressureMultiThreadedTests.java b/server/src/test/java/org/opensearch/index/ShardIndexingPressureMultiThreadedTests.java new file mode 100644 index 0000000000000..57b4fcff3f5fa --- /dev/null +++ b/server/src/test/java/org/opensearch/index/ShardIndexingPressureMultiThreadedTests.java @@ -0,0 +1,989 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index; + +import org.opensearch.action.admin.indices.stats.CommonStatsFlags; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.stats.IndexingPressurePerShardStats; +import org.opensearch.index.stats.IndexingPressureStats; +import org.opensearch.index.stats.ShardIndexingPressureStats; +import org.opensearch.test.OpenSearchTestCase; +import org.hamcrest.Matchers; + +import java.util.concurrent.atomic.AtomicInteger; + +public class ShardIndexingPressureMultiThreadedTests extends OpenSearchTestCase { + + private final Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 100) + .build(); + + final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + final ClusterService clusterService = new ClusterService(settings, clusterSettings, null); + + public void testCoordinatingPrimaryThreadedUpdateToShardLimits() throws Exception { + final int NUM_THREADS = scaledRandomIntBetween(100, 500); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + if(randomBoolean){ + releasables[counter] = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 15, false); + } else { + releasables[counter] = shardIndexingPressure.markPrimaryOperationStarted(shardId1, 15, false); + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + if(randomBoolean) { + assertEquals(NUM_THREADS * 15, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentCoordinatingBytes()); + } else { + assertEquals(NUM_THREADS * 15, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryBytes()); + } + assertEquals(NUM_THREADS * 15, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertTrue((double) (NUM_THREADS * 15) / shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits() < 0.95); + assertTrue((double) (NUM_THREADS * 15) / shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits() > 0.75); + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + + if(randomBoolean) { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCoordinatingBytes()); + } else { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryBytes()); + } + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testReplicaThreadedUpdateToShardLimits() throws Exception { + final int NUM_THREADS = scaledRandomIntBetween(100, 500); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + releasables[counter] = shardIndexingPressure.markReplicaOperationStarted(shardId1, 15, false); + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + assertEquals(NUM_THREADS * 15, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentReplicaBytes()); + assertTrue((double)(NUM_THREADS * 15) / shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentReplicaLimits() < 0.95); + assertTrue((double)(NUM_THREADS * 15) / shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentReplicaLimits() > 0.75); + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes()); + assertEquals(15, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits()); + } + + public void testCoordinatingPrimaryThreadedSimultaneousUpdateToShardLimits() throws Exception { + final int NUM_THREADS = scaledRandomIntBetween(100, 500); + final Thread[] threads = new Thread[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + for (int i = 0; i < NUM_THREADS; i++) { + threads[i] = new Thread(() -> { + if(randomBoolean) { + Releasable coodinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 100, false); + coodinating.close(); + } else { + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId1, 100, false); + primary.close(); + } + }); + try { + Thread.sleep(randomIntBetween(5, 15)); + } catch (InterruptedException e) { + //Do Nothing + } + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + if(randomBoolean) { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCoordinatingBytes()); + } else { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryBytes()); + } + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testReplicaThreadedSimultaneousUpdateToShardLimits() throws Exception { + final int NUM_THREADS = scaledRandomIntBetween(100, 500); + final Thread[] threads = new Thread[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + for (int i = 0; i < NUM_THREADS; i++) { + threads[i] = new Thread(() -> { + Releasable coodinating = shardIndexingPressure.markReplicaOperationStarted(shardId1, 100, false); + coodinating.close(); + }); + try { + Thread.sleep(randomIntBetween(5, 15)); + } catch (InterruptedException e) { + //Do Nothing + } + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes()); + assertEquals(15, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits()); + } + + public void testCoordinatingPrimaryThreadedUpdateToShardLimitsWithRandomBytes() throws Exception { + final int NUM_THREADS = scaledRandomIntBetween(100, 400); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + if(randomBoolean) { + releasables[counter] = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, + scaledRandomIntBetween(1, 20), false); + } else { + releasables[counter] = shardIndexingPressure.markPrimaryOperationStarted(shardId1, + scaledRandomIntBetween(1, 20), false); + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + + if(randomBoolean) { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCoordinatingBytes()); + } else { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryBytes()); + } + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testReplicaThreadedUpdateToShardLimitsWithRandomBytes() throws Exception { + final int NUM_THREADS = scaledRandomIntBetween(100, 400); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + releasables[counter] = shardIndexingPressure.markReplicaOperationStarted(shardId1, + scaledRandomIntBetween(1, 20), false); + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes()); + assertEquals(15, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits()); + } + + public void testCoordinatingPrimaryThreadedUpdateToShardLimitsAndRejections() throws Exception { + final int NUM_THREADS = 100; + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + AtomicInteger rejectionCount = new AtomicInteger(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + try { + if(randomBoolean) { + releasables[counter] = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 200, false); + } else { + releasables[counter] = shardIndexingPressure.markPrimaryOperationStarted(shardId1, 200, false); + } + } catch (OpenSearchRejectedExecutionException e) { + rejectionCount.addAndGet(1); + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + IndexingPressureStats nodeStats = indexingPressure.stats(); + ShardIndexingPressureStats shardStats = shardIndexingPressure.stats(); + if(randomBoolean) { + assertEquals(rejectionCount.get(), nodeStats.getCoordinatingRejections()); + assertTrue(shardStats.getIndexingPressureShardStats(shardId1).getCurrentCoordinatingBytes() < 50 * 200); + } else { + assertTrue(shardStats.getIndexingPressureShardStats(shardId1).getCurrentPrimaryBytes() < 50 * 200); + assertEquals(rejectionCount.get(), nodeStats.getPrimaryRejections()); + } + assertTrue(nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes() < 50 * 200); + assertTrue(shardStats.getIndexingPressureShardStats(shardId1).getCurrentCombinedCoordinatingAndPrimaryBytes() < 50 * 200); + + for (int i = 0; i < NUM_THREADS - rejectionCount.get(); i++) { + releasables[i].close(); + } + + nodeStats = indexingPressure.stats(); + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + shardStats = shardIndexingPressure.coldStats(); + if(randomBoolean) { + assertEquals(rejectionCount.get(), nodeStats.getCoordinatingRejections()); + assertEquals(rejectionCount.get(), shardStats.getIndexingPressureShardStats(shardId1) + .getCoordinatingNodeLimitsBreachedRejections()); + assertEquals(0, shardStats.getIndexingPressureShardStats(shardId1).getCurrentCoordinatingBytes()); + } else { + assertEquals(rejectionCount.get(), nodeStats.getPrimaryRejections()); + assertEquals(rejectionCount.get(), shardStats.getIndexingPressureShardStats(shardId1) + .getPrimaryNodeLimitsBreachedRejections()); + assertEquals(0, shardStats.getIndexingPressureShardStats(shardId1).getCurrentPrimaryBytes()); + } + + assertEquals(0, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(0, shardStats.getIndexingPressureShardStats(shardId1).getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardStats.getIndexingPressureShardStats(shardId1).getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testReplicaThreadedUpdateToShardLimitsAndRejections() throws Exception { + final int NUM_THREADS = 100; + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + AtomicInteger rejectionCount = new AtomicInteger(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + try { + releasables[counter] = shardIndexingPressure.markReplicaOperationStarted(shardId1, 300, false); + } catch (OpenSearchRejectedExecutionException e) { + rejectionCount.addAndGet(1); + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(rejectionCount.get(), nodeStats.getReplicaRejections()); + assertTrue(nodeStats.getCurrentReplicaBytes() < 50 * 300); + + ShardIndexingPressureStats shardStats = shardIndexingPressure.stats(); + assertTrue(shardStats.getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes() < 50 * 300); + + for (int i = 0; i < releasables.length - 1; i++) { + if(releasables[i] != null) { + releasables[i].close(); + } + } + + nodeStats = indexingPressure.stats(); + assertEquals(rejectionCount.get(), nodeStats.getReplicaRejections()); + assertEquals(0, nodeStats.getCurrentReplicaBytes()); + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + + shardStats = shardIndexingPressure.coldStats(); + assertEquals(rejectionCount.get(), shardStats.getIndexingPressureShardStats(shardId1) + .getReplicaNodeLimitsBreachedRejections()); + assertEquals(0, shardStats.getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes()); + assertEquals(15, shardStats.getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits()); + } + + public void testCoordinatingPrimaryConcurrentUpdatesOnShardIndexingPressureTrackerObjects() throws Exception { + final int NUM_THREADS = scaledRandomIntBetween(100, 400); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "new_uuid"); + ShardId shardId1 = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + if(randomBoolean) { + releasables[counter] = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, + scaledRandomIntBetween(1, 20), false); + } else { + releasables[counter] = shardIndexingPressure.markPrimaryOperationStarted(shardId1, + scaledRandomIntBetween(1, 20), false); + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats() + .getIndexingPressureShardStats(shardId1); + assertThat(shardStoreStats.getCurrentPrimaryAndCoordinatingLimits(), Matchers.greaterThan(100L)); + + CommonStatsFlags statsFlag = new CommonStatsFlags(); + statsFlag.includeAllShardIndexingPressureTrackers(true); + IndexingPressurePerShardStats shardStoreStats2 = shardIndexingPressure.stats(statsFlag) + .getIndexingPressureShardStats(shardId1);; + assertEquals(shardStoreStats.getCurrentPrimaryAndCoordinatingLimits(), shardStoreStats2 + .getCurrentPrimaryAndCoordinatingLimits()); + + statsFlag.includeOnlyTopIndexingPressureMetrics(true); + assertNull(shardIndexingPressure.stats(statsFlag).getIndexingPressureShardStats(shardId1)); + statsFlag.includeOnlyTopIndexingPressureMetrics(false); + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + //No object in host store as no active shards + shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + + if(randomBoolean) { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCoordinatingBytes()); + } else { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryBytes()); + } + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits()); + + shardStoreStats2 = shardIndexingPressure.stats(statsFlag).getIndexingPressureShardStats(shardId1); + assertEquals(shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits(), + shardStoreStats2.getCurrentPrimaryAndCoordinatingLimits()); + + statsFlag.includeAllShardIndexingPressureTrackers(false); + assertNull(shardIndexingPressure.stats(statsFlag).getIndexingPressureShardStats(shardId1)); + } + + public void testReplicaConcurrentUpdatesOnShardIndexingPressureTrackerObjects() throws Exception { + final int NUM_THREADS = scaledRandomIntBetween(100, 400); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "new_uuid"); + ShardId shardId1 = new ShardId(index, 0); + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + releasables[counter] = shardIndexingPressure.markReplicaOperationStarted(shardId1, + scaledRandomIntBetween(1, 20), false); + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats() + .getIndexingPressureShardStats(shardId1); + assertThat(shardStoreStats.getCurrentReplicaLimits(), Matchers.greaterThan(100L)); + + CommonStatsFlags statsFlag = new CommonStatsFlags(); + statsFlag.includeAllShardIndexingPressureTrackers(true); + IndexingPressurePerShardStats shardStoreStats2 = shardIndexingPressure.stats(statsFlag) + .getIndexingPressureShardStats(shardId1);; + assertEquals(shardStoreStats.getCurrentReplicaLimits(), shardStoreStats2.getCurrentReplicaLimits()); + + statsFlag.includeOnlyTopIndexingPressureMetrics(true); + assertNull(shardIndexingPressure.stats(statsFlag).getIndexingPressureShardStats(shardId1)); + statsFlag.includeOnlyTopIndexingPressureMetrics(false); + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + //No object in host store as no active shards + shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1); + assertNull(shardStoreStats); + + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes()); + assertEquals(15, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits()); + + shardStoreStats2 = shardIndexingPressure.stats(statsFlag).getIndexingPressureShardStats(shardId1);; + assertEquals(shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits(), + shardStoreStats2.getCurrentReplicaLimits()); + + statsFlag.includeAllShardIndexingPressureTrackers(false); + assertNull(shardIndexingPressure.stats(statsFlag).getIndexingPressureShardStats(shardId1)); + } + + public void testCoordinatingPrimaryThreadedThroughputDegradationAndRejection() throws Exception { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "15KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 100) + .build(); + final int NUM_THREADS = scaledRandomIntBetween(100, 120); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + + //Generating a load to have a fair throughput + for (int i = 0; i < NUM_THREADS; i++) { + threads[i] = new Thread(() -> { + for (int j = 0; j < randomIntBetween(400, 500); j++) { + Releasable releasable; + if(randomBoolean) { + releasable = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 100, false); + } else { + releasable = shardIndexingPressure.markPrimaryOperationStarted(shardId1, 100, false); + } + try { + Thread.sleep(100); + } catch (Exception e) { + //Do Nothing + } + releasable.close(); + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + //Generating a load to such that the requests in the window shows degradation in throughput. + for (int i = 0; i < ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.get(settings).intValue(); i++) { + int counter = i; + threads[i] = new Thread(() -> { + if(randomBoolean) { + releasables[counter] = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 100, false); + } else { + releasables[counter] = shardIndexingPressure.markPrimaryOperationStarted(shardId1, 100, false); + } + try { + Thread.sleep(200); + } catch (Exception e) { + //Do Nothing + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + for (int i = 0; i < ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.get(settings).intValue(); i++) { + releasables[i].close(); + } + + //Generate a load which breaches both primary parameter + if(randomBoolean) { + expectThrows(OpenSearchRejectedExecutionException.class, + () -> shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 11 * 1024, false)); + + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCoordinatingBytes()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingRejections()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingNodeLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingLastSuccessfulRequestLimitsBreachedRejections()); + } else { + expectThrows(OpenSearchRejectedExecutionException.class, + () -> shardIndexingPressure.markPrimaryOperationStarted(shardId1, 11 * 1024, false)); + + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryBytes()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryRejections()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryNodeLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryLastSuccessfulRequestLimitsBreachedRejections()); + } + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(15, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testReplicaThreadedThroughputDegradationAndRejection() throws Exception { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 100) + .build(); + final int NUM_THREADS = scaledRandomIntBetween(100, 120); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + + //Generating a load to have a fair throughput + for (int i = 0; i < NUM_THREADS; i++) { + threads[i] = new Thread(() -> { + for (int j = 0; j < randomIntBetween(400, 500); j++) { + Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId1, 100, false); + try { + Thread.sleep(100); + } catch (Exception e) { + //Do Nothing + } + replica.close(); + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + //Generating a load to such that the requests in the window shows degradation in throughput. + for (int i = 0; i < ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.get(settings).intValue(); i++) { + int counter = i; + threads[i] = new Thread(() -> { + releasables[counter] = shardIndexingPressure.markReplicaOperationStarted(shardId1, 100, false); + try { + Thread.sleep(200); + } catch (Exception e) { + //Do Nothing + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + for (int i = 0; i < ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.get(settings).intValue(); i++) { + releasables[i].close(); + } + + //Generate a load which breaches both primary parameter + expectThrows(OpenSearchRejectedExecutionException.class, + () -> shardIndexingPressure.markReplicaOperationStarted(shardId1, 11 * 1024, false)); + + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes()); + assertEquals(15, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getReplicaRejections()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getReplicaThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getReplicaNodeLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getReplicaLastSuccessfulRequestLimitsBreachedRejections()); + } + + public void testCoordinatingPrimaryThreadedLastSuccessfulRequestsAndRejection() throws Exception { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "250KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 100) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .build(); + final int NUM_THREADS = scaledRandomIntBetween(100, 150); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + + //One request being successful + if(randomBoolean) { + Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 10, false); + coordinating.close(); + } else { + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId1, 10, false); + primary.close(); + } + + //Generating a load such that requests are blocked requests. + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + if(randomBoolean) { + releasables[counter] = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 10, false); + } else { + releasables[counter] = shardIndexingPressure.markPrimaryOperationStarted(shardId1, 10, false); + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + //Mimic the time elapsed after requests being stuck + Thread.sleep(randomIntBetween(50, 100)); + + //Generate a load which breaches both primary parameter + if(randomBoolean) { + expectThrows(OpenSearchRejectedExecutionException.class, + () -> shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 200 * 1024, false)); + } else { + expectThrows(OpenSearchRejectedExecutionException.class, + () -> shardIndexingPressure.markPrimaryOperationStarted(shardId1, 200 * 1024, false)); + } + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + if(randomBoolean) { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCoordinatingBytes()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingNodeLimitsBreachedRejections()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingLastSuccessfulRequestLimitsBreachedRejections()); + } else { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryBytes()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryNodeLimitsBreachedRejections()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryLastSuccessfulRequestLimitsBreachedRejections()); + } + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(256, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testReplicaThreadedLastSuccessfulRequestsAndRejection() throws Exception { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "250KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 100) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .build(); + final int NUM_THREADS = scaledRandomIntBetween(100, 150); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + + //One request being successful + Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId1, 10, false); + replica.close(); + + //Generating a load such that requests are blocked requests. + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + releasables[counter] = shardIndexingPressure.markReplicaOperationStarted(shardId1, 10, false); + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + //Mimic the time elapsed after requests being stuck + Thread.sleep(randomIntBetween(50, 100)); + + //Generate a load which breaches both primary parameter + expectThrows(OpenSearchRejectedExecutionException.class, + () -> shardIndexingPressure.markReplicaOperationStarted(shardId1, 300 * 1024, false)); + + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getReplicaRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getReplicaThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getReplicaNodeLimitsBreachedRejections()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getReplicaLastSuccessfulRequestLimitsBreachedRejections()); + assertEquals(384, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits()); + } + + public void testCoordinatingPrimaryThreadedNodeLimitsAndRejection() throws Exception { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "250KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 100) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .build(); + final int NUM_THREADS = scaledRandomIntBetween(100, 150); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + + //Generating a load to such that the requests in the window shows degradation in throughput. + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + if(randomBoolean) { + releasables[counter] = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 10, false); + } else { + releasables[counter] = shardIndexingPressure.markPrimaryOperationStarted(shardId1, 10, false); + } + try { + Thread.sleep(randomIntBetween(50, 100)); + } catch (Exception e) { + //Do Nothing + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + //Generate a load which breaches both primary parameter + if(randomBoolean) { + expectThrows(OpenSearchRejectedExecutionException.class, + () -> shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 240 * 1024, false)); + } else { + expectThrows(OpenSearchRejectedExecutionException.class, + () -> shardIndexingPressure.markPrimaryOperationStarted(shardId1, 240 * 1024, false)); + } + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + if(randomBoolean) { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCoordinatingBytes()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingThroughputDegradationLimitsBreachedRejections()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingNodeLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCoordinatingLastSuccessfulRequestLimitsBreachedRejections()); + } else { + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryBytes()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryThroughputDegradationLimitsBreachedRejections()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryNodeLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getPrimaryLastSuccessfulRequestLimitsBreachedRejections()); + } + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(256, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testReplicaThreadedNodeLimitsAndRejection() throws Exception { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "250KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 100) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .build(); + final int NUM_THREADS = scaledRandomIntBetween(100, 150); + final Thread[] threads = new Thread[NUM_THREADS]; + final Releasable[] releasables = new Releasable[NUM_THREADS]; + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + + //One request being successful + Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId1, 10, false); + replica.close(); + + //Generating a load to such that the requests in the window shows degradation in throughput. + for (int i = 0; i < NUM_THREADS; i++) { + int counter = i; + threads[i] = new Thread(() -> { + releasables[counter] = shardIndexingPressure.markReplicaOperationStarted(shardId1, 10, false); + try { + Thread.sleep(randomIntBetween(50, 100)); + } catch (Exception e) { + //Do Nothing + } + }); + threads[i].start(); + } + + for (Thread t : threads) { + t.join(); + } + + //Generate a load which breaches both primary parameter + expectThrows(OpenSearchRejectedExecutionException.class, + () -> shardIndexingPressure.markReplicaOperationStarted(shardId1, 340 * 1024, false)); + + + for (int i = 0; i < NUM_THREADS; i++) { + releasables[i].close(); + } + + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getReplicaRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getReplicaThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getReplicaNodeLimitsBreachedRejections()); + assertEquals(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1) + .getReplicaLastSuccessfulRequestLimitsBreachedRejections()); + assertEquals(384, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits()); + } + +} diff --git a/server/src/test/java/org/opensearch/index/ShardIndexingPressureTests.java b/server/src/test/java/org/opensearch/index/ShardIndexingPressureTests.java new file mode 100644 index 0000000000000..2afd17396b858 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/ShardIndexingPressureTests.java @@ -0,0 +1,812 @@ +/* + * Copyright OpenSearch Contributors. + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.index; + +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.stats.IndexingPressurePerShardStats; +import org.opensearch.index.stats.IndexingPressureStats; +import org.opensearch.test.OpenSearchTestCase; + +public class ShardIndexingPressureTests extends OpenSearchTestCase { + + private final Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 100) + .build(); + + final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + final ClusterService clusterService = new ClusterService(settings, clusterSettings, null); + + public void testMemoryBytesMarkedAndReleased() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 10, false); + Releasable coordinating2 = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 50, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 15, true); + Releasable primary2 = shardIndexingPressure.markPrimaryOperationStarted(shardId, 5, false); + Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 25, true); + Releasable replica2 = shardIndexingPressure.markReplicaOperationStarted(shardId, 10, false)) { + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(60, nodeStats.getCurrentCoordinatingBytes()); + assertEquals(20, nodeStats.getCurrentPrimaryBytes()); + assertEquals(80, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(35, nodeStats.getCurrentReplicaBytes()); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertEquals(60, shardStats.getCurrentCoordinatingBytes()); + assertEquals(20, shardStats.getCurrentPrimaryBytes()); + assertEquals(80, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(35, shardStats.getCurrentReplicaBytes()); + + } + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(0, nodeStats.getCurrentCoordinatingBytes()); + assertEquals(0, nodeStats.getCurrentPrimaryBytes()); + assertEquals(0, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(0, nodeStats.getCurrentReplicaBytes()); + assertEquals(60, nodeStats.getTotalCoordinatingBytes()); + assertEquals(20, nodeStats.getTotalPrimaryBytes()); + assertEquals(80, nodeStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + assertEquals(35, nodeStats.getTotalReplicaBytes()); + + IndexingPressurePerShardStats shardHotStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertNull(shardHotStoreStats); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(60, shardStats.getTotalCoordinatingBytes()); + assertEquals(20, shardStats.getTotalPrimaryBytes()); + assertEquals(80, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + assertEquals(35, shardStats.getTotalReplicaBytes()); + } + + public void testAvoidDoubleAccounting() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 10, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(shardId, 15)) { + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(10, nodeStats.getCurrentCoordinatingBytes()); + assertEquals(15, nodeStats.getCurrentPrimaryBytes()); + assertEquals(10, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertEquals(10, shardStats.getCurrentCoordinatingBytes()); + assertEquals(15, shardStats.getCurrentPrimaryBytes()); + assertEquals(10, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + } + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(0, nodeStats.getCurrentCoordinatingBytes()); + assertEquals(0, nodeStats.getCurrentPrimaryBytes()); + assertEquals(0, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, nodeStats.getTotalCoordinatingBytes()); + assertEquals(15, nodeStats.getTotalPrimaryBytes()); + assertEquals(10, nodeStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertNull(shardStoreStats); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardStats.getTotalCoordinatingBytes()); + assertEquals(15, shardStats.getTotalPrimaryBytes()); + assertEquals(10, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + } + + public void testCoordinatingPrimaryRejections() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1024 * 3, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1024 * 3, false); + Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 1024 * 3, false)) { + if (randomBoolean()) { + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markCoordinatingOperationStarted(shardId, 1024 * 2, false)); + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(1, nodeStats.getCoordinatingRejections()); + assertEquals(1024 * 6, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertEquals(1, shardStats.getCoordinatingRejections()); + assertEquals(1024 * 6, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(1, shardStats.getCoordinatingNodeLimitsBreachedRejections()); + } else { + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markPrimaryOperationStarted(shardId, 1024 * 2, false)); + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(1, nodeStats.getPrimaryRejections()); + assertEquals(1024 * 6, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.stats() + .getIndexingPressureShardStats(shardId); + assertEquals(1, shardStats.getPrimaryRejections()); + assertEquals(1024 * 6, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(1, shardStats.getPrimaryNodeLimitsBreachedRejections()); + } + long preForceRejections = indexingPressure.stats().getPrimaryRejections(); + long preForcedShardRejections = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getPrimaryRejections(); + // Primary can be forced + Releasable forced = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1024 * 2, true); + assertEquals(preForceRejections, indexingPressure.stats().getPrimaryRejections()); + assertEquals(1024 * 8, indexingPressure.stats().getCurrentCombinedCoordinatingAndPrimaryBytes()); + + assertEquals(preForcedShardRejections, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getPrimaryRejections()); + assertEquals(1024 * 8, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(preForcedShardRejections, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getPrimaryNodeLimitsBreachedRejections()); + forced.close(); + + // Local to coordinating node primary actions not rejected + IndexingPressureStats preLocalNodeStats = indexingPressure.stats(); + IndexingPressurePerShardStats preLocalShardStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + Releasable local = shardIndexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(shardId, 1024 * 2); + assertEquals(preLocalNodeStats.getPrimaryRejections(), indexingPressure.stats().getPrimaryRejections()); + assertEquals(1024 * 6, indexingPressure.stats().getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(preLocalNodeStats.getCurrentPrimaryBytes() + 1024 * 2, indexingPressure.stats().getCurrentPrimaryBytes()); + + assertEquals(preLocalShardStats.getPrimaryRejections(), shardIndexingPressure.stats() + .getIndexingPressureShardStats(shardId).getPrimaryRejections()); + assertEquals(1024 * 6, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(preLocalShardStats.getCurrentPrimaryBytes() + 1024 * 2, shardIndexingPressure.stats() + .getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes()); + assertEquals(preLocalShardStats.getPrimaryNodeLimitsBreachedRejections(), shardIndexingPressure.stats() + .getIndexingPressureShardStats(shardId).getPrimaryNodeLimitsBreachedRejections()); + local.close(); + } + + assertEquals(1024 * 8, indexingPressure.stats().getTotalCombinedCoordinatingAndPrimaryBytes()); + assertNull(shardIndexingPressure.stats().getIndexingPressureShardStats(shardId)); + assertEquals(1024 * 8, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId) + .getTotalCombinedCoordinatingAndPrimaryBytes()); + } + + public void testReplicaRejections() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1024 * 3, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1024 * 3, false); + Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 1024 * 3, false)) { + // Replica will not be rejected until replica bytes > 15KB + Releasable replica2 = shardIndexingPressure.markReplicaOperationStarted(shardId, 1024 * 9, false); + assertEquals(1024 * 12, indexingPressure.stats().getCurrentReplicaBytes()); + assertEquals(1024 * 12, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes()); + // Replica will be rejected once we cross 15KB Shard Limit + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markReplicaOperationStarted(shardId, 1024 * 2, false)); + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(1, nodeStats.getReplicaRejections()); + assertEquals(1024 * 12, nodeStats.getCurrentReplicaBytes()); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertEquals(1, shardStats.getReplicaRejections()); + assertEquals(1024 * 12, shardStats.getCurrentReplicaBytes()); + assertEquals(1, shardStats.getReplicaNodeLimitsBreachedRejections()); + + // Replica can be forced + Releasable forced = shardIndexingPressure.markReplicaOperationStarted(shardId, 1024 * 2, true); + assertEquals(1, indexingPressure.stats().getReplicaRejections()); + assertEquals(1024 * 14, indexingPressure.stats().getCurrentReplicaBytes()); + + assertEquals(1, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getReplicaRejections()); + assertEquals(1024 * 14, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes()); + assertEquals(1, shardStats.getReplicaNodeLimitsBreachedRejections()); + forced.close(); + + replica2.close(); + } + + assertEquals(1024 * 14, indexingPressure.stats().getTotalReplicaBytes()); + assertNull(shardIndexingPressure.stats().getIndexingPressureShardStats(shardId)); + assertEquals(1024 * 14, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId).getTotalReplicaBytes()); + } + + public void testCoordinatingPrimaryShardLimitIncrease() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 2, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 2, false)) { + assertEquals(2, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes()); + assertEquals(4, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentPrimaryAndCoordinatingLimits()); // Base Limit + if (randomBoolean) { + Releasable coordinating1 = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 6, false); + assertEquals(8, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes()); + assertEquals(10, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(11, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentPrimaryAndCoordinatingLimits()); // Increased Limit + coordinating1.close(); + } else { + Releasable primary1 = shardIndexingPressure.markPrimaryOperationStarted(shardId, 6, false); + assertEquals(8, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes()); + assertEquals(10, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(11, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentPrimaryAndCoordinatingLimits()); // Increased Limit + primary1.close(); + } + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertNull(shardStoreStats); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + if(randomBoolean){ + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(8, shardStats.getTotalCoordinatingBytes()); + } else { + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(8, shardStats.getTotalPrimaryBytes()); + } + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardStats.getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testReplicaShardLimitIncrease() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 2, false)) { + assertEquals(2, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes()); + assertEquals(15, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentReplicaLimits()); // Base Limit + + Releasable replica1 = shardIndexingPressure.markReplicaOperationStarted(shardId, 14, false); + assertEquals(16, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes()); + assertEquals(18, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentReplicaLimits()); // Increased Limit + replica1.close(); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertNull(shardStoreStats); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(16, shardStats.getTotalReplicaBytes()); + assertEquals(15, shardStats.getCurrentReplicaLimits()); + } + + public void testCoordinatingPrimaryShardLimitIncreaseEvaluateSecondaryParam() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 4 * 1024, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 4 * 1024, false)) { + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCoordinatingBytes()); + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentPrimaryBytes()); + assertEquals(8 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals((long)(8*1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertNull(shardStoreStats); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(4 * 1024, shardStats.getTotalCoordinatingBytes()); + assertEquals(4 * 1024, shardStats.getTotalPrimaryBytes()); + assertEquals(8 * 1024, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardStats.getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testReplicaShardLimitIncreaseEvaluateSecondaryParam() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 11 * 1024, false)) { + assertEquals(11 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes()); + assertEquals((long)(11 * 1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentReplicaLimits()); + } + + IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.stats().getIndexingPressureShardStats(shardId); + assertNull(shardStoreStats); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(11 * 1024, shardStats.getTotalReplicaBytes()); + assertEquals(15, shardStats.getCurrentReplicaLimits()); + } + + public void testCoordinatingPrimaryShardRejectionViaSuccessfulRequestsParam() throws InterruptedException { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .build(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1 * 1024, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1 * 1024, false)) { + assertEquals(1 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes()); + assertEquals(1 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes()); + assertEquals(2 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals((long)(2*1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(1 * 1024, shardStats.getTotalCoordinatingBytes()); + assertEquals(1 * 1024, shardStats.getTotalPrimaryBytes()); + assertEquals(2 * 1024, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardStats.getCurrentPrimaryAndCoordinatingLimits()); + + Thread.sleep(25); + //Total Bytes are 9*1024 and node limit is 10*1024 + if(randomBoolean) { + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 7 * 1024, false); + Releasable coordinating1 = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1 * 1024, false)) { + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markCoordinatingOperationStarted(shardId, 1 * 1024, false)); + } + } else { + try (Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 7 * 1024, false); + Releasable primary1 = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1 * 1024, false)) { + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markPrimaryOperationStarted(shardId, 1 * 1024, false)); + } + } + + shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + if(randomBoolean) { + assertEquals(1, shardStats.getCoordinatingRejections()); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(1, shardStats.getCoordinatingLastSuccessfulRequestLimitsBreachedRejections()); + } else { + assertEquals(1, shardStats.getPrimaryRejections()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(1, shardStats.getPrimaryLastSuccessfulRequestLimitsBreachedRejections()); + } + IndexingPressureStats nodeStats = indexingPressure.stats(); + if(randomBoolean) { + assertEquals(1, nodeStats.getCoordinatingRejections()); + assertEquals(0, nodeStats.getCurrentCoordinatingBytes()); + } else { + assertEquals(1, nodeStats.getPrimaryRejections()); + assertEquals(0, nodeStats.getCurrentPrimaryBytes()); + } + } + + public void testReplicaShardRejectionViaSuccessfulRequestsParam() throws InterruptedException { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .build(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 1 * 1024, false)) { + assertEquals(1 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes()); + assertEquals((long)(1*1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentReplicaLimits()); + } + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(1 * 1024, shardStats.getTotalReplicaBytes()); + assertEquals(15, shardStats.getCurrentReplicaLimits()); + + Thread.sleep(25); + //Total Bytes are 14*1024 and node limit is 15*1024 + try (Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 10 * 1024, false); + Releasable replica1 = shardIndexingPressure.markReplicaOperationStarted(shardId, 2 * 1024, false)) { + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markReplicaOperationStarted(shardId, 2 * 1024, false)); + } + + shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(1, shardStats.getReplicaRejections()); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(1, shardStats.getReplicaLastSuccessfulRequestLimitsBreachedRejections()); + + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(1, nodeStats.getReplicaRejections()); + assertEquals(0, nodeStats.getCurrentReplicaBytes()); + } + + public void testCoordinatingPrimaryShardRejectionSkippedInShadowModeViaSuccessfulRequestsParam() throws InterruptedException { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), false) + .build(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1 * 1024, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1 * 1024, false)) { + assertEquals(1 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes()); + assertEquals(1 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes()); + assertEquals(2 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals((long)(2*1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(1 * 1024, shardStats.getTotalCoordinatingBytes()); + assertEquals(1 * 1024, shardStats.getTotalPrimaryBytes()); + assertEquals(2 * 1024, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardStats.getCurrentPrimaryAndCoordinatingLimits()); + + Thread.sleep(25); + //Total Bytes are 9*1024 and node limit is 10*1024 + if(randomBoolean) { + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 7 * 1024, false); + Releasable coordinating1 = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1 * 1024, false)) { + Releasable coordinating2 = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1 * 1024, false); + coordinating2.close(); + } + } else { + try (Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 7 * 1024, false); + Releasable primary1 = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1 * 1024, false)) { + Releasable primary2 = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1 * 1024, false); + primary2.close(); + } + } + + shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + if(randomBoolean) { + assertEquals(0, shardStats.getCoordinatingRejections()); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(1, shardStats.getCoordinatingLastSuccessfulRequestLimitsBreachedRejections()); + } else { + assertEquals(0, shardStats.getPrimaryRejections()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(1, shardStats.getPrimaryLastSuccessfulRequestLimitsBreachedRejections()); + } + IndexingPressureStats nodeStats = indexingPressure.stats(); + if(randomBoolean) { + assertEquals(0, nodeStats.getCoordinatingRejections()); + assertEquals(0, nodeStats.getCurrentCoordinatingBytes()); + } else { + assertEquals(0, nodeStats.getPrimaryRejections()); + assertEquals(0, nodeStats.getCurrentPrimaryBytes()); + } + } + + public void testReplicaShardRejectionSkippedInShadowModeViaSuccessfulRequestsParam() throws InterruptedException { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 1) + .put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), 20) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), false) + .build(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 1 * 1024, false)) { + assertEquals(1 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes()); + assertEquals((long)(1*1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentReplicaLimits()); + } + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(1 * 1024, shardStats.getTotalReplicaBytes()); + assertEquals(15, shardStats.getCurrentReplicaLimits()); + + Thread.sleep(25); + //Total Bytes are 14*1024 and node limit is 15*1024 + try (Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 10 * 1024, false); + Releasable replica1 = shardIndexingPressure.markReplicaOperationStarted(shardId, 2 * 1024, false)) { + Releasable replica2 = shardIndexingPressure.markReplicaOperationStarted(shardId, 2 * 1024, false); + replica2.close(); + } + + shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getReplicaRejections()); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(1, shardStats.getReplicaLastSuccessfulRequestLimitsBreachedRejections()); + + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(0, nodeStats.getReplicaRejections()); + assertEquals(0, nodeStats.getCurrentReplicaBytes()); + } + + public void testCoordinatingPrimaryShardRejectionViaThroughputDegradationParam() throws InterruptedException { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 1) + .build(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1 * 1024, false); + Releasable coordinating1 = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 3 * 1024, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1 * 1024, false); + Releasable primary1 = shardIndexingPressure.markPrimaryOperationStarted(shardId, 3 * 1024, false)) { + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes()); + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes()); + assertEquals(8 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals((long)(8*1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentPrimaryAndCoordinatingLimits()); + //Adding delay in the current in flight request to mimic throughput degradation + Thread.sleep(100); + } + if(randomBoolean) { + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markCoordinatingOperationStarted(shardId, 8 * 1024, false)); + } else { + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markPrimaryOperationStarted(shardId, 8 * 1024, false)); + } + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + if(randomBoolean) { + assertEquals(1, shardStats.getCoordinatingRejections()); + assertEquals(1, shardStats.getCoordinatingThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(4 * 1024, shardStats.getTotalCoordinatingBytes()); + } else { + assertEquals(1, shardStats.getPrimaryRejections()); + assertEquals(1, shardStats.getPrimaryThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(4 * 1024, shardStats.getTotalPrimaryBytes()); + } + + assertEquals(10, shardStats.getCurrentPrimaryAndCoordinatingLimits()); + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(8 * 1024, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + + IndexingPressureStats nodeStats = indexingPressure.stats(); + if(randomBoolean) { + assertEquals(1, nodeStats.getCoordinatingRejections()); + assertEquals(0, nodeStats.getCurrentCoordinatingBytes()); + } else { + assertEquals(1, nodeStats.getPrimaryRejections()); + assertEquals(0, nodeStats.getCurrentPrimaryBytes()); + } + } + + public void testReplicaShardRejectionViaThroughputDegradationParam() throws InterruptedException { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 1) + .build(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 1 * 1024, false); + Releasable replica1 = shardIndexingPressure.markReplicaOperationStarted(shardId, 3 * 1024, false)) { + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes()); + assertEquals((long)(4*1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentReplicaLimits()); + //Adding delay in the current in flight request to mimic throughput degradation + Thread.sleep(100); + } + + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markReplicaOperationStarted(shardId, 12 * 1024, false)); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(1, shardStats.getReplicaRejections()); + assertEquals(1, shardStats.getReplicaThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(4 * 1024, shardStats.getTotalReplicaBytes()); + assertEquals(15, shardStats.getCurrentReplicaLimits()); + + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(1, nodeStats.getReplicaRejections()); + assertEquals(0, nodeStats.getCurrentReplicaBytes()); + } + + public void testCoordinatingPrimaryShardRejectionSkippedInShadowModeViaThroughputDegradationParam() throws InterruptedException { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), false) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 1) + .build(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + boolean randomBoolean = randomBoolean(); + try (Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1 * 1024, false); + Releasable coordinating1 = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 3 * 1024, false); + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1 * 1024, false); + Releasable primary1 = shardIndexingPressure.markPrimaryOperationStarted(shardId, 3 * 1024, false)) { + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes()); + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes()); + assertEquals(8 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals((long)(8*1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentPrimaryAndCoordinatingLimits()); + //Adding delay in the current in flight request to mimic throughput degradation + Thread.sleep(100); + } + if(randomBoolean) { + Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 8 * 1024, false); + coordinating.close(); + } else { + Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, 8 * 1024, false); + primary.close(); + } + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + if(randomBoolean) { + assertEquals(0, shardStats.getCoordinatingRejections()); + assertEquals(1, shardStats.getCoordinatingThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(12 * 1024, shardStats.getTotalCoordinatingBytes()); + } else { + assertEquals(0, shardStats.getPrimaryRejections()); + assertEquals(1, shardStats.getPrimaryThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(12 * 1024, shardStats.getTotalPrimaryBytes()); + } + + assertEquals(10, shardStats.getCurrentPrimaryAndCoordinatingLimits()); + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(16 * 1024, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + + IndexingPressureStats nodeStats = indexingPressure.stats(); + if(randomBoolean) { + assertEquals(0, nodeStats.getCoordinatingRejections()); + assertEquals(0, nodeStats.getCurrentCoordinatingBytes()); + } else { + assertEquals(0, nodeStats.getPrimaryRejections()); + assertEquals(0, nodeStats.getCurrentPrimaryBytes()); + } + } + + public void testReplicaShardRejectionSkippedInShadowModeViaThroughputDegradationParam() throws InterruptedException { + Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB") + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), false) + .put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1) + .put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 1) + .build(); + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + try (Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 1 * 1024, false); + Releasable replica1 = shardIndexingPressure.markReplicaOperationStarted(shardId, 3 * 1024, false)) { + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes()); + assertEquals((long)(4*1024/0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId) + .getCurrentReplicaLimits()); + //Adding delay in the current in flight request to mimic throughput degradation + Thread.sleep(100); + } + + Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 12 * 1024, false); + replica.close(); + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId); + assertEquals(0, shardStats.getReplicaRejections()); + assertEquals(1, shardStats.getReplicaThroughputDegradationLimitsBreachedRejections()); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(16 * 1024, shardStats.getTotalReplicaBytes()); + assertEquals(15, shardStats.getCurrentReplicaLimits()); + + IndexingPressureStats nodeStats = indexingPressure.stats(); + assertEquals(0, nodeStats.getReplicaRejections()); + assertEquals(0, nodeStats.getCurrentReplicaBytes()); + } + + public void testShardLimitIncreaseMultipleShards() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId1 = new ShardId(index, 0); + ShardId shardId2 = new ShardId(index, 1); + try (Releasable coordinating1 = shardIndexingPressure.markCoordinatingOperationStarted(shardId1, 4 * 1024, false); + Releasable coordinating2 = shardIndexingPressure.markCoordinatingOperationStarted(shardId2, 4 * 1024, false);) { + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentCoordinatingBytes()); + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals((long)(4 * 1024 / 0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId1) + .getCurrentPrimaryAndCoordinatingLimits()); + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId2) + .getCurrentCoordinatingBytes()); + assertEquals(4 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId2) + .getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals((long)(4 * 1024 / 0.85), shardIndexingPressure.stats().getIndexingPressureShardStats(shardId2) + .getCurrentPrimaryAndCoordinatingLimits()); + } + + IndexingPressurePerShardStats shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(4 * 1024, shardStats.getTotalCoordinatingBytes()); + assertEquals(4 * 1024, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardStats.getCurrentPrimaryAndCoordinatingLimits()); + + shardStats = shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId2); + assertEquals(0, shardStats.getCurrentCoordinatingBytes()); + assertEquals(0, shardStats.getCurrentPrimaryBytes()); + assertEquals(0, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes()); + assertEquals(0, shardStats.getCurrentReplicaBytes()); + assertEquals(4 * 1024, shardStats.getTotalCoordinatingBytes()); + assertEquals(4 * 1024, shardStats.getTotalCombinedCoordinatingAndPrimaryBytes()); + assertEquals(10, shardStats.getCurrentPrimaryAndCoordinatingLimits()); + } + + public void testForceExecutionOnCoordinating() { + IndexingPressure indexingPressure = new IndexingPressure(settings, clusterService); + ShardIndexingPressure shardIndexingPressure = indexingPressure.getShardIndexingPressure(); + Index index = new Index("IndexName", "UUID"); + ShardId shardId = new ShardId(index, 0); + expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure + .markCoordinatingOperationStarted(shardId,1024 * 11, false)); + try (Releasable ignore = shardIndexingPressure.markCoordinatingOperationStarted(shardId,11 * 1024, true)) { + assertEquals(11 * 1024, shardIndexingPressure.stats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes()); + } + assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes()); + } +} diff --git a/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseSyncActionTests.java b/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseSyncActionTests.java index d56fa9529df5a..be973ef9dcc0c 100644 --- a/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseSyncActionTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseSyncActionTests.java @@ -107,7 +107,7 @@ public void testRetentionLeaseSyncActionOnPrimary() { threadPool, shardStateAction, new ActionFilters(Collections.emptySet()), - new IndexingPressure(Settings.EMPTY), + new IndexingPressure(Settings.EMPTY, clusterService), new SystemIndices(emptyMap())); final RetentionLeases retentionLeases = mock(RetentionLeases.class); final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases); @@ -145,7 +145,7 @@ public void testRetentionLeaseSyncActionOnReplica() throws Exception { threadPool, shardStateAction, new ActionFilters(Collections.emptySet()), - new IndexingPressure(Settings.EMPTY), + new IndexingPressure(Settings.EMPTY, clusterService), new SystemIndices(emptyMap())); final RetentionLeases retentionLeases = mock(RetentionLeases.class); final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases); @@ -186,7 +186,7 @@ public void testBlocks() { threadPool, shardStateAction, new ActionFilters(Collections.emptySet()), - new IndexingPressure(Settings.EMPTY), + new IndexingPressure(Settings.EMPTY, clusterService), new SystemIndices(emptyMap())); assertNull(action.indexBlockLevel()); diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 0e181ddaa42dc..8221ddaf0618b 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -1559,7 +1559,7 @@ public void onFailure(final Exception e) { threadPool, shardStateAction, actionFilters, - new IndexingPressure(settings), + new IndexingPressure(settings, clusterService), new SystemIndices(emptyMap()))), new GlobalCheckpointSyncAction( settings, @@ -1586,7 +1586,7 @@ allocationService, new AliasValidator(), shardLimitValidator, environment, index mappingUpdatedAction.setClient(client); final TransportShardBulkAction transportShardBulkAction = new TransportShardBulkAction(settings, transportService, clusterService, indicesService, threadPool, shardStateAction, mappingUpdatedAction, new UpdateHelper(scriptService), - actionFilters, new IndexingPressure(settings), new SystemIndices(emptyMap())); + actionFilters, new IndexingPressure(settings, clusterService), new SystemIndices(emptyMap())); actions.put(BulkAction.INSTANCE, new TransportBulkAction(threadPool, transportService, clusterService, new IngestService( @@ -1595,7 +1595,7 @@ clusterService, indicesService, threadPool, shardStateAction, mappingUpdatedActi Collections.emptyList(), client), transportShardBulkAction, client, actionFilters, indexNameExpressionResolver, new AutoCreateIndex(settings, clusterSettings, indexNameExpressionResolver, new SystemIndices(emptyMap())), - new IndexingPressure(settings), + new IndexingPressure(settings, clusterService), new SystemIndices(emptyMap()) )); final RestoreService restoreService = new RestoreService(