Add Shard Level Indexing Pressure (#1336)

Shard level indexing pressure improves the current Indexing Pressure framework which performs memory accounting at node level and rejects the requests. This takes a step further to have rejections based on the memory accounting at shard level along with other key performance factors like throughput and last successful requests. 

**Key features**
- Granular tracking of indexing tasks performance, at every shard level, for each node role i.e. coordinator, primary and replica.
- Smarter rejections by discarding the requests intended only for problematic index or shard, while still allowing others to continue (fairness in rejection).
- Rejections thresholds governed by combination of configurable parameters (such as memory limits on node) and dynamic parameters (such as latency increase, throughput degradation).
- Node level and shard level indexing pressure statistics exposed through stats api.
- Integration of Indexing pressure stats with Plugins for for metric visibility and auto-tuning in future.
- Control knobs to tune to the key performance thresholds which control rejections, to address any specific requirement or issues.
- Control knobs to run the feature in shadow-mode or enforced-mode. In shadow-mode only internal rejection breakdown metrics will be published while no actual rejections will be performed.

The changes were divided into small manageable chunks as part of the following PRs against a feature branch.

- Add Shard Indexing Pressure Settings. #716
- Add Shard Indexing Pressure Tracker. #717
- Refactor IndexingPressure to allow extension. #718
- Add Shard Indexing Pressure Store #838
- Add Shard Indexing Pressure Memory Manager #945
- Add ShardIndexingPressure framework level construct and Stats #1015
- Add Indexing Pressure Service which acts as orchestrator for IP #1084
- Add plumbing logic for IndexingPressureService in Transport Actions. #1113
- Add shard indexing pressure metric/stats via rest end point. #1171
- Add shard indexing pressure integration tests. #1198

Signed-off-by: Saurabh Singh <sisurab@amazon.com>
Co-authored-by: Saurabh Singh <sisurab@amazon.com>
Co-authored-by: Rabi Panda <adnapibar@gmail.com>
This commit is contained in:
Saurabh Singh 2021-10-07 23:36:15 +05:30 committed by GitHub
parent 808dbfd2ec
commit 3665daf5d0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
51 changed files with 7295 additions and 103 deletions

View File

@ -0,0 +1,297 @@
/*
* Copyright OpenSearch Contributors.
* SPDX-License-Identifier: Apache-2.0
*/
package org.opensearch.http;
import org.opensearch.client.Request;
import org.opensearch.client.Response;
import org.opensearch.client.ResponseException;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.xcontent.XContentHelper;
import org.opensearch.common.xcontent.json.JsonXContent;
import org.opensearch.index.IndexingPressure;
import org.opensearch.index.ShardIndexingPressureSettings;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.XContentTestUtils;
import java.util.ArrayList;
import java.util.Map;
import static org.opensearch.rest.RestStatus.CREATED;
import static org.opensearch.rest.RestStatus.OK;
import static org.opensearch.rest.RestStatus.TOO_MANY_REQUESTS;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.lessThan;
/**
* Test Shard Indexing Pressure Metrics and Statistics
*/
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.SUITE, supportsDedicatedMasters = false, numDataNodes = 2,
numClientNodes = 0)
public class ShardIndexingPressureRestIT extends HttpSmokeTestCase {
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "1KB")
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true)
.build();
}
@SuppressWarnings("unchecked")
public void testShardIndexingPressureStats() throws Exception {
Request createRequest = new Request("PUT", "/index_name");
createRequest.setJsonEntity("{\"settings\": {\"index\": {\"number_of_shards\": 1, \"number_of_replicas\": 1, " +
"\"write.wait_for_active_shards\": 2}}}");
final Response indexCreatedResponse = getRestClient().performRequest(createRequest);
assertThat(indexCreatedResponse.getStatusLine().getStatusCode(), equalTo(OK.getStatus()));
Request successfulIndexingRequest = new Request("POST", "/index_name/_doc/");
successfulIndexingRequest.setJsonEntity("{\"x\": \"small text\"}");
final Response indexSuccessFul = getRestClient().performRequest(successfulIndexingRequest);
assertThat(indexSuccessFul.getStatusLine().getStatusCode(), equalTo(CREATED.getStatus()));
Request getShardStats1 = new Request("GET", "/_nodes/stats/shard_indexing_pressure?include_all");
final Response shardStats1 = getRestClient().performRequest(getShardStats1);
Map<String, Object> shardStatsMap1 = XContentHelper.convertToMap(JsonXContent.jsonXContent, shardStats1.getEntity().getContent(),
true);
ArrayList<Object> values1 = new ArrayList<>(((Map<Object, Object>) shardStatsMap1.get("nodes")).values());
assertThat(values1.size(), equalTo(2));
XContentTestUtils.JsonMapView node1 = new XContentTestUtils.JsonMapView((Map<String, Object>) values1.get(0));
ArrayList<Object> shard1IndexingPressureValues = new ArrayList<>(((Map<Object, Object>) node1.get("shard_indexing_pressure.stats"))
.values());
assertThat(shard1IndexingPressureValues.size(), equalTo(1));
XContentTestUtils.JsonMapView shard1 = new XContentTestUtils.JsonMapView((Map<String, Object>) shard1IndexingPressureValues.get(0));
Integer node1TotalLimitsRejections = node1.get("shard_indexing_pressure.total_rejections_breakup.node_limits");
Integer shard1CoordinatingBytes = shard1.get("memory.total.coordinating_in_bytes");
Integer shard1PrimaryBytes = shard1.get("memory.total.primary_in_bytes");
Integer shard1ReplicaBytes = shard1.get("memory.total.replica_in_bytes");
Integer shard1CoordinatingRejections = shard1.get("rejection.coordinating.coordinating_rejections");
Integer shard1PrimaryRejections = shard1.get("rejection.primary.primary_rejections");
Integer shard1CoordinatingNodeRejections = shard1.get("rejection.coordinating.breakup.node_limits");
XContentTestUtils.JsonMapView node2 = new XContentTestUtils.JsonMapView((Map<String, Object>) values1.get(1));
ArrayList<Object> shard2IndexingPressureValues = new ArrayList<>(((Map<Object, Object>) node2.get("shard_indexing_pressure.stats"))
.values());
assertThat(shard2IndexingPressureValues.size(), equalTo(1));
XContentTestUtils.JsonMapView shard2 = new XContentTestUtils.JsonMapView((Map<String, Object>) shard2IndexingPressureValues.get(0));
Integer node2TotalLimitsRejections = node2.get("shard_indexing_pressure.total_rejections_breakup.node_limits");
Integer shard2CoordinatingBytes = shard2.get("memory.total.coordinating_in_bytes");
Integer shard2PrimaryBytes = shard2.get("memory.total.primary_in_bytes");
Integer shard2ReplicaBytes = shard2.get("memory.total.replica_in_bytes");
Integer shard2CoordinatingRejections = shard2.get("rejection.coordinating.coordinating_rejections");
Integer shard2PrimaryRejections = shard2.get("rejection.primary.primary_rejections");
Integer shard2CoordinatingNodeRejections = shard2.get("rejection.coordinating.breakup.node_limits");
if (shard1CoordinatingBytes == 0) {
assertThat(shard2CoordinatingBytes, greaterThan(0));
assertThat(shard2CoordinatingBytes, lessThan(1024));
} else {
assertThat(shard1CoordinatingBytes, greaterThan(0));
assertThat(shard1CoordinatingBytes, lessThan(1024));
}
if (shard1ReplicaBytes == 0) {
assertThat(shard1PrimaryBytes, greaterThan(0));
assertThat(shard1PrimaryBytes, lessThan(1024));
assertThat(shard2ReplicaBytes, greaterThan(0));
assertThat(shard2ReplicaBytes, lessThan(1024));
} else {
assertThat(shard2PrimaryBytes, greaterThan(0));
assertThat(shard2PrimaryBytes, lessThan(1024));
assertThat(shard2ReplicaBytes, equalTo(0));
assertThat(shard1ReplicaBytes, lessThan(1024));
}
assertThat(shard1CoordinatingRejections, equalTo(0));
assertThat(shard1PrimaryRejections, equalTo(0));
assertThat(shard2CoordinatingRejections, equalTo(0));
assertThat(shard2PrimaryRejections, equalTo(0));
assertThat(shard1CoordinatingNodeRejections, equalTo(0));
assertThat(node1TotalLimitsRejections, equalTo(0));
assertThat(node2TotalLimitsRejections, equalTo(0));
Request getNodeStats = new Request("GET", "/_nodes/stats/indexing_pressure");
final Response nodeStats = getRestClient().performRequest(getNodeStats);
Map<String, Object> nodeStatsMap = XContentHelper.convertToMap(JsonXContent.jsonXContent, nodeStats.getEntity().getContent(), true);
values1 = new ArrayList<>(((Map<Object, Object>) nodeStatsMap.get("nodes")).values());
assertThat(values1.size(), equalTo(2));
node1 = new XContentTestUtils.JsonMapView((Map<String, Object>) values1.get(0));
Integer node1CombinedBytes = node1.get("indexing_pressure.memory.total.combined_coordinating_and_primary_in_bytes");
Integer node1PrimaryBytes = node1.get("indexing_pressure.memory.total.primary_in_bytes");
Integer node1ReplicaBytes = node1.get("indexing_pressure.memory.total.replica_in_bytes");
Integer node1CoordinatingRejections = node1.get("indexing_pressure.memory.total.coordinating_rejections");
Integer node1PrimaryRejections = node1.get("indexing_pressure.memory.total.primary_rejections");
node2 = new XContentTestUtils.JsonMapView((Map<String, Object>) values1.get(1));
Integer node2IndexingBytes = node2.get("indexing_pressure.memory.total.combined_coordinating_and_primary_in_bytes");
Integer node2PrimaryBytes = node2.get("indexing_pressure.memory.total.primary_in_bytes");
Integer node2ReplicaBytes = node2.get("indexing_pressure.memory.total.replica_in_bytes");
Integer node2CoordinatingRejections = node2.get("indexing_pressure.memory.total.coordinating_rejections");
Integer node2PrimaryRejections = node2.get("indexing_pressure.memory.total.primary_rejections");
if (node1CombinedBytes == 0) {
assertThat(node2IndexingBytes, greaterThan(0));
assertThat(node2IndexingBytes, lessThan(1024));
} else {
assertThat(node1CombinedBytes, greaterThan(0));
assertThat(node1CombinedBytes, lessThan(1024));
}
if (node1ReplicaBytes == 0) {
assertThat(node1PrimaryBytes, greaterThan(0));
assertThat(node1PrimaryBytes, lessThan(1024));
assertThat(node2ReplicaBytes, greaterThan(0));
assertThat(node2ReplicaBytes, lessThan(1024));
} else {
assertThat(node2PrimaryBytes, greaterThan(0));
assertThat(node2PrimaryBytes, lessThan(1024));
assertThat(node2ReplicaBytes, equalTo(0));
assertThat(node1ReplicaBytes, lessThan(1024));
}
assertThat(node1CoordinatingRejections, equalTo(0));
assertThat(node1PrimaryRejections, equalTo(0));
assertThat(node2CoordinatingRejections, equalTo(0));
assertThat(node2PrimaryRejections, equalTo(0));
Request failedIndexingRequest = new Request("POST", "/index_name/_doc/");
String largeString = randomAlphaOfLength(10000);
failedIndexingRequest.setJsonEntity("{\"x\": " + largeString + "}");
ResponseException exception = expectThrows(ResponseException.class, () -> getRestClient().performRequest(failedIndexingRequest));
assertThat(exception.getResponse().getStatusLine().getStatusCode(), equalTo(TOO_MANY_REQUESTS.getStatus()));
Request getShardStats2 = new Request("GET", "/_nodes/stats/shard_indexing_pressure?include_all");
final Response shardStats2 = getRestClient().performRequest(getShardStats2);
Map<String, Object> shardStatsMap2 = XContentHelper.convertToMap(JsonXContent.jsonXContent, shardStats2.getEntity().getContent(),
true);
ArrayList<Object> values2 = new ArrayList<>(((Map<Object, Object>) shardStatsMap2.get("nodes")).values());
assertThat(values2.size(), equalTo(2));
XContentTestUtils.JsonMapView node1AfterRejection = new XContentTestUtils.JsonMapView((Map<String, Object>) values2.get(0));
ArrayList<Object> shard1IndexingPressureValuesAfterRejection = new ArrayList<>(((Map<Object, Object>) node1AfterRejection
.get("shard_indexing_pressure.stats")).values());
assertThat(shard1IndexingPressureValuesAfterRejection.size(), equalTo(1));
XContentTestUtils.JsonMapView shard1AfterRejection = new XContentTestUtils.JsonMapView((Map<String, Object>)
shard1IndexingPressureValuesAfterRejection.get(0));
node1TotalLimitsRejections = node1AfterRejection.get("shard_indexing_pressure.total_rejections_breakup.node_limits");
shard1CoordinatingRejections = shard1AfterRejection.get("rejection.coordinating.coordinating_rejections");
shard1PrimaryRejections = shard1AfterRejection.get("rejection.primary.primary_rejections");
shard1CoordinatingNodeRejections = shard1AfterRejection.get("rejection.coordinating.breakup.node_limits");
XContentTestUtils.JsonMapView node2AfterRejection = new XContentTestUtils.JsonMapView((Map<String, Object>) values2.get(1));
ArrayList<Object> shard2IndexingPressureValuesAfterRejection = new ArrayList<>(((Map<Object, Object>) node2AfterRejection
.get("shard_indexing_pressure.stats")).values());
assertThat(shard2IndexingPressureValuesAfterRejection.size(), equalTo(1));
XContentTestUtils.JsonMapView shard2AfterRejection = new XContentTestUtils.JsonMapView((Map<String, Object>)
shard2IndexingPressureValuesAfterRejection.get(0));
node2TotalLimitsRejections = node2AfterRejection.get("shard_indexing_pressure.total_rejections_breakup.node_limits");
shard2CoordinatingRejections = shard2AfterRejection.get("rejection.coordinating.coordinating_rejections");
shard2PrimaryRejections = shard2AfterRejection.get("rejection.primary.primary_rejections");
shard2CoordinatingNodeRejections = shard2AfterRejection.get("rejection.coordinating.breakup.node_limits");
if (shard1CoordinatingRejections == 0) {
assertThat(shard2CoordinatingRejections, equalTo(1));
assertThat(shard2CoordinatingNodeRejections, equalTo(1));
assertThat(node2TotalLimitsRejections, equalTo(1));
} else {
assertThat(shard1CoordinatingRejections, equalTo(1));
assertThat(shard1CoordinatingNodeRejections, equalTo(1));
assertThat(node1TotalLimitsRejections, equalTo(1));
}
assertThat(shard1PrimaryRejections, equalTo(0));
assertThat(shard2PrimaryRejections, equalTo(0));
Request getNodeStats2 = new Request("GET", "/_nodes/stats/indexing_pressure");
final Response nodeStats2 = getRestClient().performRequest(getNodeStats2);
Map<String, Object> nodeStatsMap2 = XContentHelper.convertToMap(JsonXContent.jsonXContent, nodeStats2.getEntity().getContent(),
true);
values2 = new ArrayList<>(((Map<Object, Object>) nodeStatsMap2.get("nodes")).values());
assertThat(values2.size(), equalTo(2));
node1AfterRejection = new XContentTestUtils.JsonMapView((Map<String, Object>) values2.get(0));
node1CoordinatingRejections = node1AfterRejection.get("indexing_pressure.memory.total.coordinating_rejections");
node1PrimaryRejections = node1.get("indexing_pressure.memory.total.primary_rejections");
node2AfterRejection = new XContentTestUtils.JsonMapView((Map<String, Object>) values2.get(1));
node2CoordinatingRejections = node2AfterRejection.get("indexing_pressure.memory.total.coordinating_rejections");
node2PrimaryRejections = node2AfterRejection.get("indexing_pressure.memory.total.primary_rejections");
if (node1CoordinatingRejections == 0) {
assertThat(node2CoordinatingRejections, equalTo(1));
} else {
assertThat(node1CoordinatingRejections, equalTo(1));
}
assertThat(node1PrimaryRejections, equalTo(0));
assertThat(node2PrimaryRejections, equalTo(0));
// Update cluster setting to enable shadow mode
Request updateSettingRequest = new Request("PUT", "/_cluster/settings");
updateSettingRequest.setJsonEntity("{\"persistent\": {\"shard_indexing_pressure\": {\"enforced\": \"false\"}}}");
final Response updateSettingResponse = getRestClient().performRequest(updateSettingRequest);
assertThat(updateSettingResponse.getStatusLine().getStatusCode(), equalTo(OK.getStatus()));
Request shadowIndexingRequest = new Request("POST", "/index_name/_doc/");
shadowIndexingRequest.setJsonEntity("{\"x\": \"text\"}");
final Response shadowIndexingResponse = getRestClient().performRequest(shadowIndexingRequest);
assertThat(shadowIndexingResponse.getStatusLine().getStatusCode(), equalTo(CREATED.getStatus()));
Request getShardStats3 = new Request("GET", "/_nodes/stats/shard_indexing_pressure?include_all");
final Response shardStats3 = getRestClient().performRequest(getShardStats3);
Map<String, Object> shardStatsMap3 = XContentHelper.convertToMap(JsonXContent.jsonXContent, shardStats3.getEntity().getContent(),
true);
ArrayList<Object> values3 = new ArrayList<>(((Map<Object, Object>) shardStatsMap3.get("nodes")).values());
assertThat(values3.size(), equalTo(2));
XContentTestUtils.JsonMapView node1AfterShadowMode = new XContentTestUtils.JsonMapView((Map<String, Object>)
values3.get(0));
ArrayList<Object> shard1IndexingPressureValuesAfterShadowMode = new ArrayList<>(((Map<Object, Object>)
node1AfterShadowMode.get("shard_indexing_pressure.stats")).values());
assertThat(shard1IndexingPressureValuesAfterShadowMode.size(), equalTo(1));
XContentTestUtils.JsonMapView shard1AfterShadowMode = new XContentTestUtils.JsonMapView((Map<String, Object>)
shard1IndexingPressureValuesAfterShadowMode.get(0));
node1TotalLimitsRejections = node1AfterShadowMode.get("shard_indexing_pressure.total_rejections_breakup_shadow_mode" +
".node_limits");
shard1CoordinatingRejections = shard1AfterShadowMode.get("rejection.coordinating.coordinating_rejections");
shard1PrimaryRejections = shard1AfterShadowMode.get("rejection.primary.primary_rejections");
shard1CoordinatingNodeRejections = shard1AfterShadowMode.get("rejection.coordinating.breakup_shadow_mode.node_limits");
XContentTestUtils.JsonMapView node2AfterShadowMode = new XContentTestUtils.JsonMapView((Map<String, Object>)
values3.get(1));
ArrayList<Object> shard2IndexingPressureValuesAfterShadowMode = new ArrayList<>(((Map<Object, Object>)
node2AfterShadowMode.get("shard_indexing_pressure.stats")).values());
assertThat(shard2IndexingPressureValuesAfterShadowMode.size(), equalTo(1));
XContentTestUtils.JsonMapView shard2AfterShadowMode = new XContentTestUtils.JsonMapView((Map<String, Object>)
shard2IndexingPressureValuesAfterShadowMode.get(0));
node2TotalLimitsRejections = node2AfterShadowMode.get("shard_indexing_pressure.total_rejections_breakup_shadow_mode" +
".node_limits");
shard2CoordinatingRejections = shard2AfterShadowMode.get("rejection.coordinating.coordinating_rejections");
shard2PrimaryRejections = shard2AfterShadowMode.get("rejection.primary.primary_rejections");
shard2CoordinatingNodeRejections = shard2AfterShadowMode.get("rejection.coordinating.breakup_shadow_mode.node_limits");
if (shard1CoordinatingRejections == 0) {
assertThat(shard2CoordinatingRejections, equalTo(1));
assertThat(shard2CoordinatingNodeRejections, equalTo(1));
assertThat(node2TotalLimitsRejections, equalTo(1));
} else {
assertThat(shard1CoordinatingRejections, equalTo(1));
assertThat(shard1CoordinatingNodeRejections, equalTo(1));
assertThat(node1TotalLimitsRejections, equalTo(1));
}
assertThat(shard1PrimaryRejections, equalTo(0));
assertThat(shard2PrimaryRejections, equalTo(0));
//Reset persistent setting to clear cluster metadata
updateSettingRequest = new Request("PUT", "/_cluster/settings");
updateSettingRequest.setJsonEntity("{\"persistent\": {\"shard_indexing_pressure\": {\"enforced\": null}}}");
getRestClient().performRequest(updateSettingRequest);
}
}

View File

@ -1283,7 +1283,15 @@ public final class ClusterAllocationExplainIT extends OpenSearchIntegTestCase {
assertTrue(parser.currentName().equals("id")
|| parser.currentName().equals("name")
|| parser.currentName().equals("transport_address")
|| parser.currentName().equals("weight_ranking"));
|| parser.currentName().equals("weight_ranking")
|| parser.currentName().equals("attributes"));
// Skip past attributes object
if (parser.currentName().equals("attributes")) {
while(!parser.nextToken().equals(Token.END_OBJECT)) {
parser.nextToken();
}
break;
}
} else {
assertTrue(token.isValue());
assertNotNull(parser.text());
@ -1403,6 +1411,11 @@ public final class ClusterAllocationExplainIT extends OpenSearchIntegTestCase {
parser.nextToken();
assertNotNull(parser.text());
parser.nextToken();
assertEquals("node_attributes", parser.currentName());
// skip past node_attributes object
while (!parser.currentName().equals("node_decision")) {
parser.nextToken();
}
assertEquals("node_decision", parser.currentName());
parser.nextToken();
return nodeName;

View File

@ -74,11 +74,15 @@ public class IndexingPressureIT extends OpenSearchIntegTestCase {
private static final Settings unboundedWriteQueue = Settings.builder().put("thread_pool.write.queue_size", -1).build();
public static final Settings settings = Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false).build();
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put(unboundedWriteQueue)
.put(settings)
.build();
}
@ -148,9 +152,12 @@ public class IndexingPressureIT extends OpenSearchIntegTestCase {
final ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
replicationSendPointReached.await();
IndexingPressure primaryWriteLimits = internalCluster().getInstance(IndexingPressure.class, primaryName);
IndexingPressure replicaWriteLimits = internalCluster().getInstance(IndexingPressure.class, replicaName);
IndexingPressure coordinatingWriteLimits = internalCluster().getInstance(IndexingPressure.class, coordinatingOnlyNode);
IndexingPressure primaryWriteLimits = internalCluster()
.getInstance(IndexingPressureService.class, primaryName).getShardIndexingPressure();
IndexingPressure replicaWriteLimits = internalCluster()
.getInstance(IndexingPressureService.class, replicaName).getShardIndexingPressure();;
IndexingPressure coordinatingWriteLimits = internalCluster()
.getInstance(IndexingPressureService.class, coordinatingOnlyNode).getShardIndexingPressure();;
assertThat(primaryWriteLimits.getCurrentCombinedCoordinatingAndPrimaryBytes(), greaterThan(bulkShardRequestSize));
assertThat(primaryWriteLimits.getCurrentPrimaryBytes(), greaterThan(bulkShardRequestSize));
@ -271,9 +278,12 @@ public class IndexingPressureIT extends OpenSearchIntegTestCase {
try (Releasable replicaRelease = blockReplicas(replicaThreadPool)) {
final ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
IndexingPressure primaryWriteLimits = internalCluster().getInstance(IndexingPressure.class, primaryName);
IndexingPressure replicaWriteLimits = internalCluster().getInstance(IndexingPressure.class, replicaName);
IndexingPressure coordinatingWriteLimits = internalCluster().getInstance(IndexingPressure.class, coordinatingOnlyNode);
IndexingPressure primaryWriteLimits = internalCluster()
.getInstance(IndexingPressureService.class, primaryName).getShardIndexingPressure();
IndexingPressure replicaWriteLimits = internalCluster()
.getInstance(IndexingPressureService.class, replicaName).getShardIndexingPressure();
IndexingPressure coordinatingWriteLimits = internalCluster()
.getInstance(IndexingPressureService.class, coordinatingOnlyNode).getShardIndexingPressure();
assertBusy(() -> {
assertThat(primaryWriteLimits.getCurrentCombinedCoordinatingAndPrimaryBytes(), greaterThan(bulkShardRequestSize));
@ -335,9 +345,12 @@ public class IndexingPressureIT extends OpenSearchIntegTestCase {
try (Releasable replicaRelease = blockReplicas(replicaThreadPool)) {
final ActionFuture<BulkResponse> successFuture = client(primaryName).bulk(bulkRequest);
IndexingPressure primaryWriteLimits = internalCluster().getInstance(IndexingPressure.class, primaryName);
IndexingPressure replicaWriteLimits = internalCluster().getInstance(IndexingPressure.class, replicaName);
IndexingPressure coordinatingWriteLimits = internalCluster().getInstance(IndexingPressure.class, coordinatingOnlyNode);
IndexingPressure primaryWriteLimits = internalCluster()
.getInstance(IndexingPressureService.class, primaryName).getShardIndexingPressure();
IndexingPressure replicaWriteLimits = internalCluster()
.getInstance(IndexingPressureService.class, replicaName).getShardIndexingPressure();
IndexingPressure coordinatingWriteLimits = internalCluster()
.getInstance(IndexingPressureService.class, coordinatingOnlyNode).getShardIndexingPressure();
assertBusy(() -> {
assertThat(primaryWriteLimits.getCurrentCombinedCoordinatingAndPrimaryBytes(), greaterThan(bulkShardRequestSize));

View File

@ -0,0 +1,710 @@
/*
* Copyright OpenSearch Contributors.
* SPDX-License-Identifier: Apache-2.0
*/
package org.opensearch.index;
import org.apache.lucene.util.RamUsageEstimator;
import org.opensearch.action.ActionFuture;
import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
import org.opensearch.action.admin.indices.stats.ShardStats;
import org.opensearch.action.bulk.BulkItemRequest;
import org.opensearch.action.bulk.BulkRequest;
import org.opensearch.action.bulk.BulkShardRequest;
import org.opensearch.action.bulk.TransportShardBulkAction;
import org.opensearch.action.bulk.BulkResponse;
import org.opensearch.action.index.IndexRequest;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.cluster.routing.ShardRouting;
import org.opensearch.common.UUIDs;
import org.opensearch.common.collect.Tuple;
import org.opensearch.common.lease.Releasable;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException;
import org.opensearch.index.shard.ShardId;
import org.opensearch.indices.IndicesService;
import org.opensearch.plugins.Plugin;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.InternalSettingsPlugin;
import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.transport.MockTransportService;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.TransportService;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.CountDownLatch;
import java.util.stream.Stream;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 2, numClientNodes = 1,
transportClientRatio = 0.0D)
public class ShardIndexingPressureIT extends OpenSearchIntegTestCase {
public static final String INDEX_NAME = "test_index";
private static final Settings unboundedWriteQueue = Settings.builder().put("thread_pool.write.queue_size", -1).build();
public static final Settings settings = Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true).build();
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put(unboundedWriteQueue)
.put(settings)
.build();
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Arrays.asList(MockTransportService.TestPlugin.class, InternalSettingsPlugin.class);
}
@Override
protected int numberOfReplicas() {
return 1;
}
@Override
protected int numberOfShards() {
return 1;
}
public void testShardIndexingPressureTrackingDuringBulkWrites() throws Exception {
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String replicaName = primaryReplicaNodeNames.v2();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
final CountDownLatch replicationSendPointReached = new CountDownLatch(1);
final CountDownLatch latchBlockingReplicationSend = new CountDownLatch(1);
TransportService primaryService = internalCluster().getInstance(TransportService.class, primaryName);
final MockTransportService primaryTransportService = (MockTransportService) primaryService;
TransportService replicaService = internalCluster().getInstance(TransportService.class, replicaName);
final MockTransportService replicaTransportService = (MockTransportService) replicaService;
primaryTransportService.addSendBehavior((connection, requestId, action, request, options) -> {
if (action.equals(TransportShardBulkAction.ACTION_NAME + "[r]")) {
try {
replicationSendPointReached.countDown();
latchBlockingReplicationSend.await();
} catch (InterruptedException e) {
throw new IllegalStateException(e);
}
}
connection.sendRequest(requestId, action, request, options);
});
final ThreadPool replicaThreadPool = replicaTransportService.getThreadPool();
final Releasable replicaRelease = blockReplicas(replicaThreadPool);
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
final long bulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
try {
final ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
replicationSendPointReached.await();
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId= new ShardId(index, 0);
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTracker = internalCluster().getInstance(IndexingPressureService.class, replicaName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertThat(primaryShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes(),
equalTo(bulkShardRequestSize));
assertThat(primaryShardTracker.getPrimaryOperationTracker().getStatsTracker().getCurrentBytes(), equalTo(bulkShardRequestSize));
assertEquals(0, primaryShardTracker.getCoordinatingOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, replicaShardTracker.getCoordinatingOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getPrimaryOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(bulkShardRequestSize, coordinatingShardTracker.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(bulkShardRequestSize, coordinatingShardTracker.getCoordinatingOperationTracker().getStatsTracker()
.getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getPrimaryOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
latchBlockingReplicationSend.countDown();
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
final BulkRequest secondBulkRequest = new BulkRequest();
secondBulkRequest.add(request);
// Use the primary or the replica data node as the coordinating node this time
boolean usePrimaryAsCoordinatingNode = randomBoolean();
final ActionFuture<BulkResponse> secondFuture;
if (usePrimaryAsCoordinatingNode) {
secondFuture = client(primaryName).bulk(secondBulkRequest);
} else {
secondFuture = client(replicaName).bulk(secondBulkRequest);
}
final long secondBulkShardRequestSize = request.ramBytesUsed() + RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
if (usePrimaryAsCoordinatingNode) {
assertBusy(() -> {
assertThat(primaryShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes(),
equalTo(bulkShardRequestSize + secondBulkShardRequestSize));
assertEquals(secondBulkShardRequestSize, primaryShardTracker.getCoordinatingOperationTracker().getStatsTracker()
.getCurrentBytes());
assertThat(primaryShardTracker.getPrimaryOperationTracker().getStatsTracker().getCurrentBytes(),
equalTo(bulkShardRequestSize + secondBulkShardRequestSize));
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, replicaShardTracker.getCoordinatingOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getPrimaryOperationTracker().getStatsTracker().getCurrentBytes());
});
} else {
assertThat(primaryShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes(),
equalTo(bulkShardRequestSize));
assertEquals(secondBulkShardRequestSize, replicaShardTracker.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(secondBulkShardRequestSize, replicaShardTracker.getCoordinatingOperationTracker().getStatsTracker()
.getCurrentBytes());
assertEquals(0, replicaShardTracker.getPrimaryOperationTracker().getStatsTracker().getCurrentBytes());
}
assertEquals(bulkShardRequestSize, coordinatingShardTracker.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertBusy(() -> assertThat(replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes(),
equalTo(bulkShardRequestSize + secondBulkShardRequestSize)));
replicaRelease.close();
successFuture.actionGet();
secondFuture.actionGet();
assertEquals(0, primaryShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTracker.getCoordinatingOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, primaryShardTracker.getPrimaryOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, replicaShardTracker.getCoordinatingOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTracker.getCoordinatingOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getPrimaryOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
} finally {
if (replicationSendPointReached.getCount() > 0) {
replicationSendPointReached.countDown();
}
replicaRelease.close();
if (latchBlockingReplicationSend.getCount() > 0) {
latchBlockingReplicationSend.countDown();
}
replicaRelease.close();
primaryTransportService.clearAllRules();
}
}
public void testWritesRejectedForSingleCoordinatingShardDueToNodeLevelLimitBreach() throws Exception {
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
final long bulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
restartCluster(Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(),
(long) (bulkShardRequestSize * 1.5) + "B").build());
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String replicaName = primaryReplicaNodeNames.v2();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
final ThreadPool replicaThreadPool = internalCluster().getInstance(ThreadPool.class, replicaName);
try (Releasable replicaRelease = blockReplicas(replicaThreadPool)) {
final ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId= new ShardId(index, 0);
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTracker = internalCluster().getInstance(IndexingPressureService.class, replicaName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertBusy(() -> {
assertEquals(bulkShardRequestSize, primaryShardTracker.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(bulkShardRequestSize, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(bulkShardRequestSize, coordinatingShardTracker.getCommonOperationTracker().
getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
});
expectThrows(OpenSearchRejectedExecutionException.class, () -> {
if (randomBoolean()) {
client(coordinatingOnlyNode).bulk(bulkRequest).actionGet();
} else if (randomBoolean()) {
client(primaryName).bulk(bulkRequest).actionGet();
} else {
client(replicaName).bulk(bulkRequest).actionGet();
}
});
replicaRelease.close();
successFuture.actionGet();
assertEquals(0, primaryShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
}
}
public void testWritesRejectedFairnessWithMultipleCoordinatingShardsDueToNodeLevelLimitBreach() throws Exception {
final BulkRequest largeBulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME + "large").id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
largeBulkRequest.add(request);
}
final long largeBulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
final BulkRequest smallBulkRequest = new BulkRequest();
totalRequestSize = 0;
for (int i = 0; i < 10; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME + "small").id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(10)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
smallBulkRequest.add(request);
}
final long smallBulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 10)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
restartCluster(Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(),
(long) (largeBulkShardRequestSize * 1.5) + "B").build());
assertAcked(prepareCreate(INDEX_NAME + "large", Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME + "large");
assertAcked(prepareCreate(INDEX_NAME + "small", Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME + "small");
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME + "large");
String primaryName = primaryReplicaNodeNames.v1();
String replicaName = primaryReplicaNodeNames.v2();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
final ThreadPool replicaThreadPool = internalCluster().getInstance(ThreadPool.class, replicaName);
try (Releasable replicaRelease = blockReplicas(replicaThreadPool)) {
final ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(largeBulkRequest);
ShardId shardId = null;
for (IndexService indexService : internalCluster().getInstance(IndicesService.class, primaryName)) {
if (indexService.getIndexSettings().getIndex().getName().equals(INDEX_NAME + "large")) {
shardId = new ShardId(indexService.getIndexSettings().getIndex(), 0);
}
}
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTracker = internalCluster().getInstance(IndexingPressureService.class, replicaName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertBusy(() -> {
assertEquals(largeBulkShardRequestSize, primaryShardTracker.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(largeBulkShardRequestSize, replicaShardTracker.getReplicaOperationTracker().getStatsTracker()
.getCurrentBytes());
assertEquals(largeBulkShardRequestSize, coordinatingShardTracker.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
});
// Large request on a shard with already large occupancy is rejected
expectThrows(OpenSearchRejectedExecutionException.class, () -> {
client(coordinatingOnlyNode).bulk(largeBulkRequest).actionGet();
});
replicaRelease.close();
successFuture.actionGet();
assertEquals(0, primaryShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
// Try sending a small request now instead which should succeed one the new shard with less occupancy
final ThreadPool replicaThreadPoolSmallRequest = internalCluster().getInstance(ThreadPool.class, replicaName);
try (Releasable replicaReleaseSmallRequest = blockReplicas(replicaThreadPoolSmallRequest)) {
final ActionFuture<BulkResponse> successFutureSmallRequest = client(coordinatingOnlyNode).bulk(smallBulkRequest);
shardId = null;
for (IndexService indexService : internalCluster().getInstance(IndicesService.class, primaryName)) {
if (indexService.getIndexSettings().getIndex().getName().equals(INDEX_NAME + "small")) {
shardId = new ShardId(indexService.getIndexSettings().getIndex(), 0);
}
}
ShardIndexingPressureTracker primaryShardTrackerSmall = internalCluster().getInstance(IndexingPressureService.class,
primaryName).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTrackerSmall = internalCluster().getInstance(IndexingPressureService.class,
replicaName).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTrackerSmall = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertBusy(() -> {
assertEquals(smallBulkShardRequestSize, primaryShardTrackerSmall.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTrackerSmall.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(smallBulkShardRequestSize, replicaShardTrackerSmall.getReplicaOperationTracker().getStatsTracker()
.getCurrentBytes());
assertEquals(smallBulkShardRequestSize, coordinatingShardTrackerSmall.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
});
replicaReleaseSmallRequest.close();
successFutureSmallRequest.actionGet();
assertEquals(0, primaryShardTrackerSmall.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTrackerSmall.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, replicaShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTrackerSmall.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTrackerSmall.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
}
}
}
public void testWritesRejectedForSinglePrimaryShardDueToNodeLevelLimitBreach() throws Exception {
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
final long bulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
restartCluster(Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(),
(long) (bulkShardRequestSize * 1.5) + "B").build());
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String replicaName = primaryReplicaNodeNames.v2();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
final ThreadPool replicaThreadPool = internalCluster().getInstance(ThreadPool.class, replicaName);
try (Releasable replicaRelease = blockReplicas(replicaThreadPool)) {
final ActionFuture<BulkResponse> successFuture = client(primaryName).bulk(bulkRequest);
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId= new ShardId(index, 0);
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTracker = internalCluster().getInstance(IndexingPressureService.class, replicaName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertBusy(() -> {
assertEquals(bulkShardRequestSize, primaryShardTracker.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(bulkShardRequestSize, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
});
BulkResponse responses = client(coordinatingOnlyNode).bulk(bulkRequest).actionGet();
assertTrue(responses.hasFailures());
assertThat(responses.getItems()[0].getFailure().getCause().getCause(), instanceOf(OpenSearchRejectedExecutionException.class));
replicaRelease.close();
successFuture.actionGet();
assertEquals(0, primaryShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(1, primaryShardTracker.getPrimaryOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(0, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
}
}
public void testWritesRejectedFairnessWithMultiplePrimaryShardsDueToNodeLevelLimitBreach() throws Exception {
final BulkRequest largeBulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME + "large").id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
largeBulkRequest.add(request);
}
final long largeBulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
final BulkRequest smallBulkRequest = new BulkRequest();
totalRequestSize = 0;
for (int i = 0; i < 10; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME + "small").id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(10)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
smallBulkRequest.add(request);
}
final long smallBulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 10)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
restartCluster(Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(),
(long) (largeBulkShardRequestSize * 1.5) + "B").build());
assertAcked(prepareCreate(INDEX_NAME + "large", Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME + "large");
assertAcked(prepareCreate(INDEX_NAME + "small", Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME + "small");
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME + "large");
String primaryName = primaryReplicaNodeNames.v1();
String replicaName = primaryReplicaNodeNames.v2();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
final ThreadPool replicaThreadPool = internalCluster().getInstance(ThreadPool.class, replicaName);
try (Releasable replicaRelease = blockReplicas(replicaThreadPool)) {
final ActionFuture<BulkResponse> successFuture = client(primaryName).bulk(largeBulkRequest);
ShardId shardId = null;
for (IndexService indexService : internalCluster().getInstance(IndicesService.class, primaryName)) {
if (indexService.getIndexSettings().getIndex().getName().equals(INDEX_NAME + "large")) {
shardId = new ShardId(indexService.getIndexSettings().getIndex(), 0);
}
}
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTracker = internalCluster().getInstance(IndexingPressureService.class, replicaName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertBusy(() -> {
assertEquals(largeBulkShardRequestSize, primaryShardTracker.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(largeBulkShardRequestSize, replicaShardTracker.getReplicaOperationTracker().getStatsTracker()
.getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
});
BulkResponse responses = client(coordinatingOnlyNode).bulk(largeBulkRequest).actionGet();
assertTrue(responses.hasFailures());
assertThat(responses.getItems()[0].getFailure().getCause().getCause(), instanceOf(OpenSearchRejectedExecutionException.class));
replicaRelease.close();
successFuture.actionGet();
assertEquals(0, primaryShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(1, primaryShardTracker.getPrimaryOperationTracker().getRejectionTracker().getTotalRejections());
// Try sending a small request now instead which should succeed one the new shard with less occupancy
final ThreadPool replicaThreadPoolSmallRequest = internalCluster().getInstance(ThreadPool.class, replicaName);
try (Releasable replicaReleaseSmallRequest = blockReplicas(replicaThreadPoolSmallRequest)) {
final ActionFuture<BulkResponse> successFutureSmallRequest = client(primaryName).bulk(smallBulkRequest);
shardId = null;
for (IndexService indexService : internalCluster().getInstance(IndicesService.class, primaryName)) {
if (indexService.getIndexSettings().getIndex().getName().equals(INDEX_NAME + "small")) {
shardId = new ShardId(indexService.getIndexSettings().getIndex(), 0);
}
}
ShardIndexingPressureTracker primaryShardTrackerSmall = internalCluster().getInstance(IndexingPressureService.class,
primaryName).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTrackerSmall = internalCluster().getInstance(IndexingPressureService.class,
replicaName).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTrackerSmall = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertBusy(() -> {
assertEquals(smallBulkShardRequestSize, primaryShardTrackerSmall.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTrackerSmall.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(smallBulkShardRequestSize, replicaShardTrackerSmall.getReplicaOperationTracker().getStatsTracker()
.getCurrentBytes());
assertEquals(0, coordinatingShardTrackerSmall.getCommonOperationTracker()
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
});
replicaReleaseSmallRequest.close();
successFutureSmallRequest.actionGet();
assertEquals(0, primaryShardTrackerSmall.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, primaryShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, replicaShardTrackerSmall.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, replicaShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, coordinatingShardTrackerSmall.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0, coordinatingShardTrackerSmall.getReplicaOperationTracker().getStatsTracker().getCurrentBytes());
assertEquals(0, primaryShardTrackerSmall.getPrimaryOperationTracker().getRejectionTracker().getTotalRejections());
}
}
}
private Tuple<String, String> getPrimaryReplicaNodeNames(String indexName) {
IndicesStatsResponse response = client().admin().indices().prepareStats(indexName).get();
String primaryId = Stream.of(response.getShards())
.map(ShardStats::getShardRouting)
.filter(ShardRouting::primary)
.findAny()
.get()
.currentNodeId();
String replicaId = Stream.of(response.getShards())
.map(ShardStats::getShardRouting)
.filter(sr -> sr.primary() == false)
.findAny()
.get()
.currentNodeId();
DiscoveryNodes nodes = client().admin().cluster().prepareState().get().getState().nodes();
String primaryName = nodes.get(primaryId).getName();
String replicaName = nodes.get(replicaId).getName();
return new Tuple<>(primaryName, replicaName);
}
private String getCoordinatingOnlyNode() {
return client().admin().cluster().prepareState().get().getState().nodes().getCoordinatingOnlyNodes().iterator().next()
.value.getName();
}
private Releasable blockReplicas(ThreadPool threadPool) {
final CountDownLatch blockReplication = new CountDownLatch(1);
final int threads = threadPool.info(ThreadPool.Names.WRITE).getMax();
final CountDownLatch pointReached = new CountDownLatch(threads);
for (int i = 0; i< threads; ++i) {
threadPool.executor(ThreadPool.Names.WRITE).execute(() -> {
try {
pointReached.countDown();
blockReplication.await();
} catch (InterruptedException e) {
throw new IllegalStateException(e);
}
});
}
return () -> {
if (blockReplication.getCount() > 0) {
blockReplication.countDown();
}
};
}
private void restartCluster(Settings settings) throws Exception {
internalCluster().fullRestart(new InternalTestCluster.RestartCallback() {
@Override
public Settings onNodeStopped(String nodeName) {
return Settings.builder().put(unboundedWriteQueue).put(settings).build();
}
});
}
}

View File

@ -0,0 +1,827 @@
/*
* Copyright OpenSearch Contributors.
* SPDX-License-Identifier: Apache-2.0
*/
package org.opensearch.index;
import org.apache.lucene.util.RamUsageEstimator;
import org.opensearch.action.ActionFuture;
import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
import org.opensearch.action.admin.indices.stats.ShardStats;
import org.opensearch.action.bulk.BulkItemRequest;
import org.opensearch.action.bulk.BulkRequest;
import org.opensearch.action.bulk.BulkResponse;
import org.opensearch.action.bulk.BulkShardRequest;
import org.opensearch.action.bulk.TransportShardBulkAction;
import org.opensearch.action.index.IndexRequest;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.node.DiscoveryNodes;
import org.opensearch.cluster.routing.ShardRouting;
import org.opensearch.common.UUIDs;
import org.opensearch.common.collect.Tuple;
import org.opensearch.common.lease.Releasable;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException;
import org.opensearch.index.shard.ShardId;
import org.opensearch.indices.IndicesService;
import org.opensearch.plugins.Plugin;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.InternalSettingsPlugin;
import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.transport.MockTransportService;
import org.opensearch.threadpool.ThreadPool;
import org.opensearch.transport.TransportService;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.CountDownLatch;
import java.util.stream.Stream;
import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;
@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 2, numClientNodes = 1,
transportClientRatio = 0.0D)
public class ShardIndexingPressureSettingsIT extends OpenSearchIntegTestCase {
public static final String INDEX_NAME = "test_index";
private static final Settings unboundedWriteQueue = Settings.builder().put("thread_pool.write.queue_size", -1).build();
public static final Settings settings = Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false)
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), false)
.build();
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put(unboundedWriteQueue)
.put(settings)
.build();
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Arrays.asList(MockTransportService.TestPlugin.class, InternalSettingsPlugin.class);
}
@Override
protected int numberOfReplicas() {
return 1;
}
@Override
protected int numberOfShards() {
return 1;
}
public void testNodeAttributeSetForShardIndexingPressure() throws Exception {
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
for (DiscoveryNode nodes : client().admin().cluster().prepareState().get().getState().nodes()) {
assertEquals("true", nodes.getAttributes().get("shard_indexing_pressure_enabled"));
}
}
public void testShardIndexingPressureFeatureEnabledDisabledSetting() {
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String replicaName = primaryReplicaNodeNames.v2();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
final long bulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId = new ShardId(index, 0);
IndexingPressure primaryNodeLimits = internalCluster()
.getInstance(IndexingPressureService.class, primaryName).getShardIndexingPressure();
IndexingPressure replicaNodeLimits = internalCluster()
.getInstance(IndexingPressureService.class, replicaName).getShardIndexingPressure();
IndexingPressure coordinatingNodeLimits = internalCluster()
.getInstance(IndexingPressureService.class, coordinatingOnlyNode).getShardIndexingPressure();
// Assert no tracking at shard level for indexing pressure since default setting is disabled
ShardIndexingPressure shardIndexingPressure = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure();
assertFalse(shardIndexingPressure.isShardIndexingPressureEnabled());
assertNull(shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId));
assertEquals(bulkRequest.ramBytesUsed(), coordinatingNodeLimits.totalCoordinatingBytes.get());
assertEquals(bulkShardRequestSize, primaryNodeLimits.totalPrimaryBytes.get());
assertEquals(bulkShardRequestSize, replicaNodeLimits.totalReplicaBytes.get());
// Enable the setting for shard indexing pressure as true
ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest();
updateSettingsRequest.persistentSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
// Send a second request
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTracker = internalCluster().getInstance(IndexingPressureService.class, replicaName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertTrue(shardIndexingPressure.isShardIndexingPressureEnabled());
assertEquals(bulkShardRequestSize, coordinatingShardTracker.getCommonOperationTracker()
.getTotalCombinedCoordinatingAndPrimaryBytes());
assertEquals(bulkShardRequestSize, coordinatingShardTracker.getCoordinatingOperationTracker().getStatsTracker().getTotalBytes());
assertEquals(bulkShardRequestSize, primaryShardTracker.getCommonOperationTracker().getTotalCombinedCoordinatingAndPrimaryBytes());
assertEquals(bulkShardRequestSize, primaryShardTracker.getPrimaryOperationTracker().getStatsTracker().getTotalBytes());
assertEquals(bulkShardRequestSize, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getTotalBytes());
assertEquals(bulkRequest.ramBytesUsed() + bulkShardRequestSize, coordinatingNodeLimits.totalCoordinatingBytes.get());
assertEquals(2 * bulkShardRequestSize, primaryNodeLimits.totalPrimaryBytes.get());
assertEquals(2 * bulkShardRequestSize, replicaNodeLimits.totalReplicaBytes.get());
// Disable the setting again for shard indexing pressure as true
updateSettingsRequest = new ClusterUpdateSettingsRequest();
updateSettingsRequest.persistentSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
// Send a third request which should not be tracked since feature is disabled again
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
assertFalse(shardIndexingPressure.isShardIndexingPressureEnabled());
assertEquals(bulkShardRequestSize, coordinatingShardTracker.getCommonOperationTracker().
getTotalCombinedCoordinatingAndPrimaryBytes());
assertEquals(bulkShardRequestSize, coordinatingShardTracker.getCoordinatingOperationTracker().getStatsTracker().getTotalBytes());
assertEquals(bulkShardRequestSize, primaryShardTracker.getCommonOperationTracker().getTotalCombinedCoordinatingAndPrimaryBytes());
assertEquals(bulkShardRequestSize, primaryShardTracker.getPrimaryOperationTracker().getStatsTracker().getTotalBytes());
assertEquals(bulkShardRequestSize, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getTotalBytes());
assertEquals(2 * bulkRequest.ramBytesUsed() + bulkShardRequestSize, coordinatingNodeLimits.totalCoordinatingBytes.get());
assertEquals(3 * bulkShardRequestSize, primaryNodeLimits.totalPrimaryBytes.get());
assertEquals(3 * bulkShardRequestSize, replicaNodeLimits.totalReplicaBytes.get());
}
public void testShardIndexingPressureNodeLimitUpdateSetting() throws Exception {
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
final long bulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
// Set the Node limit threshold above the request-size; for no rejection
restartCluster(Settings.builder().put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), (long) (bulkShardRequestSize * 1.5) + "B").build());
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String replicaName = primaryReplicaNodeNames.v2();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId= new ShardId(index, 0);
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTracker = internalCluster().getInstance(IndexingPressureService.class, replicaName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
// Tracking done with no rejections
assertEquals(0, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(bulkShardRequestSize, coordinatingShardTracker.getCoordinatingOperationTracker().getStatsTracker().getTotalBytes());
assertEquals(bulkShardRequestSize, primaryShardTracker.getPrimaryOperationTracker().getStatsTracker().getTotalBytes());
assertEquals(bulkShardRequestSize, replicaShardTracker.getReplicaOperationTracker().getStatsTracker().getTotalBytes());
// Update the indexing byte setting to a lower value
restartCluster(Settings.builder().put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.put(IndexingPressure.MAX_INDEXING_BYTES.getKey(),
(long) (bulkShardRequestSize * 0.5) + "B").build());
// Any node receiving the request will end up rejecting request due to node level limit breached
expectThrows(OpenSearchRejectedExecutionException.class, () -> {
if (randomBoolean()) {
client(coordinatingOnlyNode).bulk(bulkRequest).actionGet();
} else if (randomBoolean()) {
client(primaryName).bulk(bulkRequest).actionGet();
} else {
client(replicaName).bulk(bulkRequest).actionGet();
}
});
}
public void testShardIndexingPressureEnforcedEnabledDisabledSetting() throws Exception {
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
final long bulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
Settings settings = Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true)
.put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), (long) (bulkShardRequestSize * 3.8) + "B")
.put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 1)
.put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), "10ms")
.build();
restartCluster(settings);
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId= new ShardId(index, 0);
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
// Send first request which gets successful to set the last successful time-stamp
ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
// Send couple of more requests which remains outstanding
Releasable primaryReleasable = blockPrimary(primaryName);
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
ActionFuture<BulkResponse> secondSuccessFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
// Delay to breach the success time stamp threshold
Thread.sleep(25);
// This request breaches the threshold and hence will be rejected
expectThrows(OpenSearchRejectedExecutionException.class, () -> client(coordinatingOnlyNode).bulk(bulkRequest).actionGet());
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
//Complete the outstanding requests
primaryReleasable.close();
successFuture.actionGet();
secondSuccessFuture.actionGet();
// Disable the enforced mode by setting it to false
ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest();
updateSettingsRequest.persistentSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), false));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
// Repeat the previous set of request patterns; but no actual rejection this time
// First successful request
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
// Send couple of requests which remains outstanding
primaryReleasable = blockPrimary(primaryName);
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
secondSuccessFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
// Add a delay to let next request breach the success time stamp threshold
Thread.sleep(25);
// Send the third request which will not get rejected; unlike previous time
ActionFuture<BulkResponse> thirdSuccessFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
// No new actual rejection
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
// Shadow mode rejection count breakup still updated
assertEquals(2, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
primaryReleasable.close();
successFuture.actionGet();
secondSuccessFuture.actionGet();
thirdSuccessFuture.actionGet();
// Enable the enforced mode again by setting it to true
updateSettingsRequest = new ClusterUpdateSettingsRequest();
updateSettingsRequest.persistentSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
// Repeat the previous set of requests; to get an actual rejection again this time
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
// Send couple of requests which remains outstanding
primaryReleasable = blockPrimary(primaryName);
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
secondSuccessFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
Thread.sleep(25);
// This request breaches the threshold and hence will be rejected
expectThrows(OpenSearchRejectedExecutionException.class, () -> client(coordinatingOnlyNode).bulk(bulkRequest).actionGet());
// new rejection added to the actual rejection count
assertEquals(2, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(3, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
primaryReleasable.close();
successFuture.actionGet();
secondSuccessFuture.actionGet();
}
public void testShardIndexingPressureEnforcedEnabledNoOpIfFeatureDisabled() throws Exception {
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
Settings settings = Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false)
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true)
.build();
restartCluster(settings);
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId= new ShardId(index, 0);
// Send first request
ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
// Verify No actual tracking being done
ShardIndexingPressure shardIndexingPressure = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure();
assertFalse(shardIndexingPressure.isShardIndexingPressureEnabled());
assertNull(shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId));
}
public void testShardIndexingPressureVerifyShardMinLimitSettingUpdate() throws Exception {
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
// Set the Node limit threshold above the request-size; for no rejection
long initialNodeLimit = 1000000;
restartCluster(Settings.builder().put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), (long) initialNodeLimit + "B").build());
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String replicaName = primaryReplicaNodeNames.v2();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId= new ShardId(index, 0);
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker replicaShardTracker = internalCluster().getInstance(IndexingPressureService.class, replicaName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
// Verify initial shard limits
assertEquals(1000, coordinatingShardTracker.getPrimaryAndCoordinatingLimits());
assertEquals(1000, primaryShardTracker.getPrimaryAndCoordinatingLimits());
assertEquals(1500, replicaShardTracker.getReplicaLimits());
// New Increment factor
double incrementFactor = 0.01d;
// Update the setting for initial shard limit
ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest();
updateSettingsRequest.persistentSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_MIN_LIMIT.getKey(), incrementFactor));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
// Create a new index to send new request
assertAcked(prepareCreate(INDEX_NAME + "new", Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME + "new");
primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME + "new");
primaryName = primaryReplicaNodeNames.v1();
replicaName = primaryReplicaNodeNames.v2();
coordinatingOnlyNode = getCoordinatingOnlyNode();
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
index = indexService.getIndexSettings().getIndex();
shardId= new ShardId(index, 0);
primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
replicaShardTracker = internalCluster().getInstance(IndexingPressureService.class, replicaName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class, coordinatingOnlyNode)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
// Verify updated initial shard limits
assertEquals(10000, coordinatingShardTracker.getPrimaryAndCoordinatingLimits());
assertEquals(10000, primaryShardTracker.getPrimaryAndCoordinatingLimits());
assertEquals(15000, replicaShardTracker.getReplicaLimits());
}
public void testShardIndexingPressureLastSuccessfulSettingsUpdate() throws Exception {
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
final long bulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
boolean randomBoolean = randomBoolean();
Settings settings = Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true)
.put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), (long) (bulkShardRequestSize * 3.8) + "B")
.put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 1)
.put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), "10ms")
.build();
restartCluster(settings);
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId= new ShardId(index, 0);
// Send first request which gets successful to set the last successful time-stamp
if (randomBoolean) {
ActionFuture<BulkResponse> successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
} else {
ActionFuture<BulkResponse> successFuture = client(primaryName).bulk(bulkRequest);
successFuture.actionGet();
}
// Send couple of more requests which remains outstanding to increase time-stamp value
ActionFuture<BulkResponse> successFuture;
ActionFuture<BulkResponse> secondSuccessFuture;
Releasable primaryReleasable = blockPrimary(primaryName);
if (randomBoolean) {
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
secondSuccessFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
} else {
successFuture = client(primaryName).bulk(bulkRequest);
secondSuccessFuture = client(primaryName).bulk(bulkRequest);
}
// Delay to breach the success time stamp threshold
Thread.sleep(25);
// This request breaches the threshold and hence will be rejected
if (randomBoolean) {
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
expectThrows(OpenSearchRejectedExecutionException.class, () -> client(coordinatingOnlyNode).bulk(bulkRequest).actionGet());
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
} else {
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
expectThrows(OpenSearchRejectedExecutionException.class, () -> client(primaryName).bulk(bulkRequest).actionGet());
assertEquals(1, primaryShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(1, primaryShardTracker.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
}
primaryReleasable.close();
successFuture.actionGet();
secondSuccessFuture.actionGet();
// Update the outstanding threshold setting to see no rejections
ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest();
updateSettingsRequest.persistentSettings(Settings.builder()
.put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 10));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
// Verify no rejection with similar request pattern
primaryReleasable = blockPrimary(primaryName);
if (randomBoolean) {
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
secondSuccessFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
} else {
successFuture = client(primaryName).bulk(bulkRequest);
secondSuccessFuture = client(primaryName).bulk(bulkRequest);
}
Thread.sleep(25);
ActionFuture<BulkResponse> thirdSuccessFuture;
if (randomBoolean) {
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
thirdSuccessFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
} else {
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
thirdSuccessFuture = client(primaryName).bulk(bulkRequest);
assertEquals(1, primaryShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(1, primaryShardTracker.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
}
primaryReleasable.close();
successFuture.actionGet();
secondSuccessFuture.actionGet();
thirdSuccessFuture.actionGet();
}
public void testShardIndexingPressureRequestSizeWindowSettingUpdate() throws Exception {
final BulkRequest bulkRequest = new BulkRequest();
int totalRequestSize = 0;
for (int i = 0; i < 80; ++i) {
IndexRequest request = new IndexRequest(INDEX_NAME).id(UUIDs.base64UUID())
.source(Collections.singletonMap("key", randomAlphaOfLength(50)));
totalRequestSize += request.ramBytesUsed();
assertTrue(request.ramBytesUsed() > request.source().length());
bulkRequest.add(request);
}
final long bulkShardRequestSize = totalRequestSize + (RamUsageEstimator.shallowSizeOfInstance(BulkItemRequest.class) * 80)
+ RamUsageEstimator.shallowSizeOfInstance(BulkShardRequest.class);
Settings settings = Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true)
.put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), (long) (bulkShardRequestSize * 1.2) + "B")
.put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 1)
.put(ShardIndexingPressureMemoryManager.THROUGHPUT_DEGRADATION_LIMITS.getKey(), 1)
.build();
restartCluster(settings);
assertAcked(prepareCreate(INDEX_NAME, Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)));
ensureGreen(INDEX_NAME);
Tuple<String, String> primaryReplicaNodeNames = getPrimaryReplicaNodeNames(INDEX_NAME);
String primaryName = primaryReplicaNodeNames.v1();
String replicaName = primaryReplicaNodeNames.v2();
String coordinatingOnlyNode = getCoordinatingOnlyNode();
IndexService indexService = internalCluster().getInstance(IndicesService.class, primaryName).iterator().next();
Index index = indexService.getIndexSettings().getIndex();
ShardId shardId= new ShardId(index, 0);
boolean randomBoolean = randomBoolean();
// Send first request which gets successful
ActionFuture<BulkResponse> successFuture;
if (randomBoolean) {
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
} else {
successFuture = client(primaryName).bulk(bulkRequest);
}
successFuture.actionGet();
final CountDownLatch replicationSendPointReached = new CountDownLatch(1);
final CountDownLatch latchBlockingReplicationSend = new CountDownLatch(1);
TransportService primaryService = internalCluster().getInstance(TransportService.class, primaryName);
final MockTransportService primaryTransportService = (MockTransportService) primaryService;
TransportService replicaService = internalCluster().getInstance(TransportService.class, replicaName);
final MockTransportService replicaTransportService = (MockTransportService) replicaService;
primaryTransportService.addSendBehavior((connection, requestId, action, request, options) -> {
if (action.equals(TransportShardBulkAction.ACTION_NAME + "[r]")) {
try {
replicationSendPointReached.countDown();
latchBlockingReplicationSend.await();
} catch (InterruptedException e) {
throw new IllegalStateException(e);
}
}
connection.sendRequest(requestId, action, request, options);
});
final ThreadPool replicaThreadPool = replicaTransportService.getThreadPool();
final Releasable replicaRelease = blockReplicas(replicaThreadPool);
// Send one more requests which remains outstanding for delayed time
if (randomBoolean) {
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
} else {
successFuture = client(primaryName).bulk(bulkRequest);
}
// Delay to breach the success time stamp threshold
Thread.sleep(3000);
latchBlockingReplicationSend.countDown();
replicaRelease.close();
successFuture.actionGet();
// This request breaches the threshold and hence will be rejected
if (randomBoolean) {
expectThrows(OpenSearchRejectedExecutionException.class, () -> client(coordinatingOnlyNode).bulk(bulkRequest).actionGet());
} else {
expectThrows(OpenSearchRejectedExecutionException.class, () -> client(primaryName).bulk(bulkRequest).actionGet());
}
// Update the outstanding threshold setting to see no rejections
ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest();
updateSettingsRequest.persistentSettings(Settings.builder()
.put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 10));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
// Verify no rejection with similar request pattern
if (randomBoolean) {
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
successFuture.actionGet();
successFuture = client(coordinatingOnlyNode).bulk(bulkRequest);
Thread.sleep(10);
successFuture.actionGet();
client(coordinatingOnlyNode).bulk(bulkRequest).actionGet();
ShardIndexingPressureTracker coordinatingShardTracker = internalCluster().getInstance(IndexingPressureService.class,
coordinatingOnlyNode).getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(1, coordinatingShardTracker.getCoordinatingOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
} else {
successFuture = client(primaryName).bulk(bulkRequest);
successFuture.actionGet();
successFuture = client(primaryName).bulk(bulkRequest);
Thread.sleep(10);
successFuture.actionGet();
client(primaryName).bulk(bulkRequest).actionGet();
ShardIndexingPressureTracker primaryShardTracker = internalCluster().getInstance(IndexingPressureService.class, primaryName)
.getShardIndexingPressure().getShardIndexingPressureTracker(shardId);
assertEquals(1, primaryShardTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections());
assertEquals(1, primaryShardTracker.getCoordinatingOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
}
}
private Releasable blockReplicas(ThreadPool threadPool) {
final CountDownLatch blockReplication = new CountDownLatch(1);
final int threads = threadPool.info(ThreadPool.Names.WRITE).getMax();
final CountDownLatch pointReached = new CountDownLatch(threads);
for (int i = 0; i< threads; ++i) {
threadPool.executor(ThreadPool.Names.WRITE).execute(() -> {
try {
pointReached.countDown();
blockReplication.await();
} catch (InterruptedException e) {
throw new IllegalStateException(e);
}
});
}
return () -> {
if (blockReplication.getCount() > 0) {
blockReplication.countDown();
}
};
}
private Releasable blockPrimary(String primaryName) {
final CountDownLatch latchBlockingReplicationSend = new CountDownLatch(1);
TransportService primaryService = internalCluster().getInstance(TransportService.class, primaryName);
final MockTransportService primaryTransportService = (MockTransportService) primaryService;
primaryTransportService.addSendBehavior((connection, requestId, action, request, options) -> {
if (action.equals(TransportShardBulkAction.ACTION_NAME + "[r]")) {
try {
latchBlockingReplicationSend.await();
} catch (InterruptedException e) {
throw new IllegalStateException(e);
}
}
connection.sendRequest(requestId, action, request, options);
});
return () -> {
if (latchBlockingReplicationSend.getCount() > 0) {
latchBlockingReplicationSend.countDown();
}
};
}
private Tuple<String, String> getPrimaryReplicaNodeNames(String indexName) {
IndicesStatsResponse response = client().admin().indices().prepareStats(indexName).get();
String primaryId = Stream.of(response.getShards())
.map(ShardStats::getShardRouting)
.filter(ShardRouting::primary)
.findAny()
.get()
.currentNodeId();
String replicaId = Stream.of(response.getShards())
.map(ShardStats::getShardRouting)
.filter(sr -> sr.primary() == false)
.findAny()
.get()
.currentNodeId();
DiscoveryNodes nodes = client().admin().cluster().prepareState().get().getState().nodes();
String primaryName = nodes.get(primaryId).getName();
String replicaName = nodes.get(replicaId).getName();
return new Tuple<>(primaryName, replicaName);
}
private String getCoordinatingOnlyNode() {
return client().admin().cluster().prepareState().get().getState().nodes().getCoordinatingOnlyNodes().iterator().next()
.value.getName();
}
private void restartCluster(Settings settings) throws Exception {
internalCluster().fullRestart(new InternalTestCluster.RestartCallback() {
@Override
public Settings onNodeStopped(String nodeName) {
return Settings.builder().put(unboundedWriteQueue).put(settings).build();
}
});
}
}

View File

@ -43,7 +43,9 @@ import org.opensearch.common.xcontent.ToXContentFragment;
import org.opensearch.common.xcontent.XContentBuilder;
import org.opensearch.discovery.DiscoveryStats;
import org.opensearch.http.HttpStats;
import org.opensearch.index.ShardIndexingPressureSettings;
import org.opensearch.index.stats.IndexingPressureStats;
import org.opensearch.index.stats.ShardIndexingPressureStats;
import org.opensearch.indices.NodeIndicesStats;
import org.opensearch.indices.breaker.AllCircuitBreakerStats;
import org.opensearch.ingest.IngestStats;
@ -112,6 +114,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
@Nullable
private IndexingPressureStats indexingPressureStats;
@Nullable
private ShardIndexingPressureStats shardIndexingPressureStats;
public NodeStats(StreamInput in) throws IOException {
super(in);
timestamp = in.readVLong();
@ -147,6 +152,12 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
} else {
indexingPressureStats = null;
}
if (ShardIndexingPressureSettings.isShardIndexingPressureAttributeEnabled()) {
shardIndexingPressureStats = in.readOptionalWriteable(ShardIndexingPressureStats::new);
} else {
shardIndexingPressureStats = null;
}
}
public NodeStats(DiscoveryNode node, long timestamp, @Nullable NodeIndicesStats indices,
@ -158,7 +169,8 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
@Nullable IngestStats ingestStats,
@Nullable AdaptiveSelectionStats adaptiveSelectionStats,
@Nullable ScriptCacheStats scriptCacheStats,
@Nullable IndexingPressureStats indexingPressureStats) {
@Nullable IndexingPressureStats indexingPressureStats,
@Nullable ShardIndexingPressureStats shardIndexingPressureStats) {
super(node);
this.timestamp = timestamp;
this.indices = indices;
@ -176,6 +188,7 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
this.adaptiveSelectionStats = adaptiveSelectionStats;
this.scriptCacheStats = scriptCacheStats;
this.indexingPressureStats = indexingPressureStats;
this.shardIndexingPressureStats = shardIndexingPressureStats;
}
public long getTimestamp() {
@ -280,6 +293,11 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
return indexingPressureStats;
}
@Nullable
public ShardIndexingPressureStats getShardIndexingPressureStats() {
return shardIndexingPressureStats;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
@ -309,6 +327,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_9_0)) {
out.writeOptionalWriteable(indexingPressureStats);
}
if (ShardIndexingPressureSettings.isShardIndexingPressureAttributeEnabled()) {
out.writeOptionalWriteable(shardIndexingPressureStats);
}
}
@Override
@ -378,6 +399,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
if (getIndexingPressureStats() != null) {
getIndexingPressureStats().toXContent(builder, params);
}
if (getShardIndexingPressureStats() != null) {
getShardIndexingPressureStats().toXContent(builder, params);
}
return builder;
}
}

View File

@ -238,7 +238,8 @@ public class NodesStatsRequest extends BaseNodesRequest<NodesStatsRequest> {
INGEST("ingest"),
ADAPTIVE_SELECTION("adaptive_selection"),
SCRIPT_CACHE("script_cache"),
INDEXING_PRESSURE("indexing_pressure"),;
INDEXING_PRESSURE("indexing_pressure"),
SHARD_INDEXING_PRESSURE("shard_indexing_pressure");
private String metricName;

View File

@ -97,7 +97,8 @@ public class TransportNodesStatsAction extends TransportNodesAction<NodesStatsRe
NodesStatsRequest.Metric.INGEST.containedIn(metrics),
NodesStatsRequest.Metric.ADAPTIVE_SELECTION.containedIn(metrics),
NodesStatsRequest.Metric.SCRIPT_CACHE.containedIn(metrics),
NodesStatsRequest.Metric.INDEXING_PRESSURE.containedIn(metrics));
NodesStatsRequest.Metric.INDEXING_PRESSURE.containedIn(metrics),
NodesStatsRequest.Metric.SHARD_INDEXING_PRESSURE.containedIn(metrics));
}
public static class NodeStatsRequest extends BaseNodeRequest {

View File

@ -113,7 +113,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeRequest) {
NodeInfo nodeInfo = nodeService.info(true, true, false, true, false, true, false, true, false, false, false);
NodeStats nodeStats = nodeService.stats(CommonStatsFlags.NONE,
true, true, true, false, true, false, false, false, false, false, true, false, false, false);
true, true, true, false, true, false, false, false, false, false, true, false, false, false, false);
List<ShardStats> shardsStats = new ArrayList<>();
for (IndexService indexService : indicesService) {
for (IndexShard indexShard : indexService) {

View File

@ -36,6 +36,7 @@ import org.opensearch.LegacyESVersion;
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.ShardIndexingPressureSettings;
import java.io.IOException;
import java.util.Collections;
@ -53,6 +54,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.
@ -80,6 +83,10 @@ public class CommonStatsFlags implements Writeable, Cloneable {
if (in.getVersion().onOrAfter(LegacyESVersion.V_7_2_0)) {
includeUnloadedSegments = in.readBoolean();
}
if (ShardIndexingPressureSettings.isShardIndexingPressureAttributeEnabled()) {
includeAllShardIndexingPressureTrackers = in.readBoolean();
includeOnlyTopIndexingPressureMetrics = in.readBoolean();
}
}
@Override
@ -98,6 +105,10 @@ public class CommonStatsFlags implements Writeable, Cloneable {
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_2_0)) {
out.writeBoolean(includeUnloadedSegments);
}
if (ShardIndexingPressureSettings.isShardIndexingPressureAttributeEnabled()) {
out.writeBoolean(includeAllShardIndexingPressureTrackers);
out.writeBoolean(includeOnlyTopIndexingPressureMetrics);
}
}
/**
@ -111,6 +122,8 @@ public class CommonStatsFlags implements Writeable, Cloneable {
completionDataFields = null;
includeSegmentFileSizes = false;
includeUnloadedSegments = false;
includeAllShardIndexingPressureTrackers = false;
includeOnlyTopIndexingPressureMetrics = false;
return this;
}
@ -125,6 +138,8 @@ public class CommonStatsFlags implements Writeable, Cloneable {
completionDataFields = null;
includeSegmentFileSizes = false;
includeUnloadedSegments = false;
includeAllShardIndexingPressureTrackers = false;
includeOnlyTopIndexingPressureMetrics = false;
return this;
}
@ -198,10 +213,28 @@ public class CommonStatsFlags implements Writeable, Cloneable {
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;
}

View File

@ -75,7 +75,7 @@ import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.AtomicArray;
import org.opensearch.index.Index;
import org.opensearch.index.IndexNotFoundException;
import org.opensearch.index.IndexingPressure;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.VersionType;
import org.opensearch.index.seqno.SequenceNumbers;
import org.opensearch.index.shard.ShardId;
@ -127,25 +127,26 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
private final NodeClient client;
private final IndexNameExpressionResolver indexNameExpressionResolver;
private static final String DROPPED_ITEM_WITH_AUTO_GENERATED_ID = "auto-generated";
private final IndexingPressure indexingPressure;
private final IndexingPressureService indexingPressureService;
private final SystemIndices systemIndices;
@Inject
public TransportBulkAction(ThreadPool threadPool, TransportService transportService,
ClusterService clusterService, IngestService ingestService,
TransportShardBulkAction shardBulkAction, NodeClient client, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver,
AutoCreateIndex autoCreateIndex, IndexingPressure indexingPressure, SystemIndices systemIndices) {
TransportShardBulkAction shardBulkAction, NodeClient client, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver,
AutoCreateIndex autoCreateIndex, IndexingPressureService indexingPressureService,
SystemIndices systemIndices) {
this(threadPool, transportService, clusterService, ingestService, shardBulkAction, client, actionFilters,
indexNameExpressionResolver, autoCreateIndex, indexingPressure, systemIndices, System::nanoTime);
indexNameExpressionResolver, autoCreateIndex, indexingPressureService, systemIndices, System::nanoTime);
}
public TransportBulkAction(ThreadPool threadPool, TransportService transportService,
ClusterService clusterService, IngestService ingestService,
TransportShardBulkAction shardBulkAction, NodeClient client,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
AutoCreateIndex autoCreateIndex, IndexingPressure indexingPressure, SystemIndices systemIndices,
LongSupplier relativeTimeProvider) {
AutoCreateIndex autoCreateIndex, IndexingPressureService indexingPressureService,
SystemIndices systemIndices, LongSupplier relativeTimeProvider) {
super(BulkAction.NAME, transportService, actionFilters, BulkRequest::new, ThreadPool.Names.SAME);
Objects.requireNonNull(relativeTimeProvider);
this.threadPool = threadPool;
@ -157,7 +158,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
this.ingestForwarder = new IngestActionForwarder(transportService);
this.client = client;
this.indexNameExpressionResolver = indexNameExpressionResolver;
this.indexingPressure = indexingPressure;
this.indexingPressureService = indexingPressureService;
this.systemIndices = systemIndices;
clusterService.addStateApplier(this.ingestForwarder);
}
@ -184,7 +185,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener<BulkResponse> 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 = indexingPressureService.markCoordinatingOperationStarted(indexingBytes, isOnlySystem);
final ActionListener<BulkResponse> releasingListener = ActionListener.runBefore(listener, releasable::close);
final String executorName = isOnlySystem ? Names.SYSTEM_WRITE : Names.WRITE;
try {
@ -562,7 +563,11 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
if (task != null) {
bulkShardRequest.setParentTask(nodeId, task.getId());
}
shardBulkAction.execute(bulkShardRequest, new ActionListener<BulkShardResponse>() {
// Add the shard level accounting for coordinating and supply the listener
final boolean isOnlySystem = isOnlySystem(bulkRequest, clusterService.state().metadata().getIndicesLookup(), systemIndices);
final Releasable releasable = indexingPressureService.markCoordinatingOperationStarted(shardId,
bulkShardRequest.ramBytesUsed(), isOnlySystem);
shardBulkAction.execute(bulkShardRequest, ActionListener.runBefore(new ActionListener<BulkShardResponse>() {
@Override
public void onResponse(BulkShardResponse bulkShardResponse) {
for (BulkItemResponse bulkItemResponse : bulkShardResponse.getResponses()) {
@ -584,7 +589,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
final String indexName = concreteIndices.getConcreteIndex(request.index()).getName();
DocWriteRequest<?> docWriteRequest = request.request();
responses.set(request.id(), new BulkItemResponse(request.id(), docWriteRequest.opType(),
new BulkItemResponse.Failure(indexName, docWriteRequest.type(), docWriteRequest.id(), e)));
new BulkItemResponse.Failure(indexName, docWriteRequest.type(), docWriteRequest.id(), e)));
}
if (counter.decrementAndGet() == 0) {
finishHim();
@ -595,7 +600,7 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
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
}

View File

@ -68,7 +68,7 @@ import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.xcontent.ToXContent;
import org.opensearch.common.xcontent.XContentHelper;
import org.opensearch.common.xcontent.XContentType;
import org.opensearch.index.IndexingPressure;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.engine.VersionConflictEngineException;
import org.opensearch.index.get.GetResult;
@ -115,9 +115,9 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
public TransportShardBulkAction(Settings settings, TransportService transportService, ClusterService clusterService,
IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction,
MappingUpdatedAction mappingUpdatedAction, UpdateHelper updateHelper, ActionFilters actionFilters,
IndexingPressure indexingPressure, SystemIndices systemIndices) {
IndexingPressureService indexingPressureService, SystemIndices systemIndices) {
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
BulkShardRequest::new, BulkShardRequest::new, EXECUTOR_NAME_FUNCTION, false, indexingPressure, systemIndices);
BulkShardRequest::new, BulkShardRequest::new, EXECUTOR_NAME_FUNCTION, false, indexingPressureService, systemIndices);
this.updateHelper = updateHelper;
this.mappingUpdatedAction = mappingUpdatedAction;
}

View File

@ -45,7 +45,7 @@ import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.inject.Inject;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.settings.Settings;
import org.opensearch.index.IndexingPressure;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.seqno.SequenceNumbers;
import org.opensearch.index.shard.IndexShard;
@ -80,11 +80,11 @@ public class TransportResyncReplicationAction extends TransportWriteAction<Resyn
public TransportResyncReplicationAction(Settings settings, TransportService transportService,
ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool,
ShardStateAction shardStateAction, ActionFilters actionFilters,
IndexingPressure indexingPressure, SystemIndices systemIndices) {
IndexingPressureService indexingPressureService, SystemIndices systemIndices) {
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
ResyncReplicationRequest::new, ResyncReplicationRequest::new, EXECUTOR_NAME_FUNCTION,
true, /* we should never reject resync because of thread pool capacity on primary */
indexingPressure, systemIndices);
indexingPressureService, systemIndices);
}
@Override

View File

@ -49,7 +49,7 @@ import org.opensearch.common.Nullable;
import org.opensearch.common.io.stream.Writeable;
import org.opensearch.common.lease.Releasable;
import org.opensearch.common.settings.Settings;
import org.opensearch.index.IndexingPressure;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.engine.Engine;
import org.opensearch.index.mapper.MapperParsingException;
import org.opensearch.index.shard.IndexShard;
@ -76,7 +76,7 @@ public abstract class TransportWriteAction<
Response extends ReplicationResponse & WriteResponse
> extends TransportReplicationAction<Request, ReplicaRequest, Response> {
protected final IndexingPressure indexingPressure;
protected final IndexingPressureService indexingPressureService;
protected final SystemIndices systemIndices;
private final Function<IndexShard, String> executorFunction;
@ -85,13 +85,14 @@ public abstract class TransportWriteAction<
ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool,
ShardStateAction shardStateAction, ActionFilters actionFilters, Writeable.Reader<Request> request,
Writeable.Reader<ReplicaRequest> replicaRequest, Function<IndexShard, String> executorFunction,
boolean forceExecutionOnPrimary, IndexingPressure indexingPressure, SystemIndices systemIndices) {
boolean forceExecutionOnPrimary, IndexingPressureService indexingPressureService,
SystemIndices systemIndices) {
// We pass ThreadPool.Names.SAME to the super class as we control the dispatching to the
// ThreadPool.Names.WRITE/ThreadPool.Names.SYSTEM_WRITE thread pools in this class.
super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
request, replicaRequest, ThreadPool.Names.SAME, true, forceExecutionOnPrimary);
this.executorFunction = executorFunction;
this.indexingPressure = indexingPressure;
this.indexingPressureService = indexingPressureService;
this.systemIndices = systemIndices;
}
@ -101,7 +102,7 @@ public abstract class TransportWriteAction<
@Override
protected Releasable checkOperationLimits(Request request) {
return indexingPressure.markPrimaryOperationStarted(primaryOperationSize(request), force(request));
return indexingPressureService.markPrimaryOperationStarted(request.shardId, primaryOperationSize(request), force(request));
}
protected boolean force(ReplicatedWriteRequest<?> request) {
@ -119,7 +120,8 @@ public abstract class TransportWriteAction<
// 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));
return indexingPressureService.markPrimaryOperationLocalToCoordinatingNodeStarted(request.shardId,
primaryOperationSize(request));
} else {
return () -> {};
}
@ -127,7 +129,7 @@ public abstract class TransportWriteAction<
// 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 indexingPressureService.markPrimaryOperationStarted(request.shardId, primaryOperationSize(request), force(request));
}
}
@ -137,7 +139,7 @@ public abstract class TransportWriteAction<
@Override
protected Releasable checkReplicaLimits(ReplicaRequest request) {
return indexingPressure.markReplicaOperationStarted(replicaOperationSize(request), force(request));
return indexingPressureService.markReplicaOperationStarted(request.shardId, replicaOperationSize(request), force(request));
}
protected long replicaOperationSize(ReplicaRequest request) {

View File

@ -204,6 +204,13 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements
return clusterState;
}
/**
* Returns true if the appliedClusterState is not null
*/
public boolean isInitialClusterStateSet() {
return Objects.nonNull(this.state.get());
}
/**
* Adds a high priority applier of updated cluster states.
*/

View File

@ -49,6 +49,7 @@ 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 org.opensearch.index.IndexingPressureService;
import org.opensearch.node.Node;
import org.opensearch.threadpool.ThreadPool;
@ -78,6 +79,8 @@ public class ClusterService extends AbstractLifecycleComponent {
private RerouteService rerouteService;
private IndexingPressureService indexingPressureService;
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));
@ -203,6 +206,20 @@ public class ClusterService extends AbstractLifecycleComponent {
return masterService;
}
/**
* Getter and Setter for IndexingPressureService, This method exposes IndexingPressureService stats to other plugins for usage.
* Although Indexing Pressure instances can be accessed via Node and NodeService class but none of them are
* present in the createComponents signature of Plugin interface currently. {@link org.opensearch.plugins.Plugin#createComponents}
* Going forward, IndexingPressureService will have required constructs for exposing listeners/interfaces for plugin development.(#478)
*/
public void setIndexingPressureService(IndexingPressureService indexingPressureService) {
this.indexingPressureService = indexingPressureService;
}
public IndexingPressureService getIndexingPressureService() {
return indexingPressureService;
}
public ClusterApplierService getClusterApplierService() {
return clusterApplierService;
}

View File

@ -34,6 +34,12 @@ package org.opensearch.common.settings;
import org.apache.logging.log4j.LogManager;
import org.opensearch.action.main.TransportMainAction;
import org.opensearch.cluster.routing.allocation.decider.NodeLoadAwareAllocationDecider;
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.watcher.ResourceWatcherService;
import org.opensearch.action.admin.cluster.configuration.TransportAddVotingConfigExclusionsAction;
import org.opensearch.action.admin.indices.close.TransportCloseIndexAction;
@ -101,9 +107,6 @@ import org.opensearch.gateway.DanglingIndicesState;
import org.opensearch.gateway.GatewayService;
import org.opensearch.gateway.PersistedClusterStateService;
import org.opensearch.http.HttpTransportSettings;
import org.opensearch.index.IndexModule;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.IndexingPressure;
import org.opensearch.indices.IndexingMemoryController;
import org.opensearch.indices.IndicesQueryCache;
import org.opensearch.indices.IndicesRequestCache;
@ -585,7 +588,19 @@ public final class ClusterSettings extends AbstractScopedSettings {
IndexingPressure.MAX_INDEXING_BYTES,
NodeLoadAwareAllocationDecider.CLUSTER_ROUTING_ALLOCATION_LOAD_AWARENESS_PROVISIONED_CAPACITY_SETTING,
NodeLoadAwareAllocationDecider.CLUSTER_ROUTING_ALLOCATION_LOAD_AWARENESS_SKEW_FACTOR_SETTING,
NodeLoadAwareAllocationDecider.CLUSTER_ROUTING_ALLOCATION_LOAD_AWARENESS_ALLOW_UNASSIGNED_PRIMARIES_SETTING)));
NodeLoadAwareAllocationDecider.CLUSTER_ROUTING_ALLOCATION_LOAD_AWARENESS_ALLOW_UNASSIGNED_PRIMARIES_SETTING,
ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED,
ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED,
ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW,
ShardIndexingPressureSettings.SHARD_MIN_LIMIT,
ShardIndexingPressureStore.MAX_COLD_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)));
public static List<SettingUpgrader<?>> BUILT_IN_SETTING_UPGRADERS = Collections.unmodifiableList(Arrays.asList(
SniffConnectionStrategy.SEARCH_REMOTE_CLUSTER_SEEDS_UPGRADER,

View File

@ -34,6 +34,7 @@ package org.opensearch.index;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.opensearch.common.inject.Inject;
import org.opensearch.common.lease.Releasable;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
@ -51,23 +52,24 @@ public class IndexingPressure {
private static final Logger logger = LogManager.getLogger(IndexingPressure.class);
private final AtomicLong currentCombinedCoordinatingAndPrimaryBytes = new AtomicLong(0);
private final AtomicLong currentCoordinatingBytes = new AtomicLong(0);
private final AtomicLong currentPrimaryBytes = new AtomicLong(0);
private final AtomicLong currentReplicaBytes = new AtomicLong(0);
protected final AtomicLong currentCombinedCoordinatingAndPrimaryBytes = new AtomicLong(0);
protected final AtomicLong currentCoordinatingBytes = new AtomicLong(0);
protected final AtomicLong currentPrimaryBytes = new AtomicLong(0);
protected final AtomicLong currentReplicaBytes = new AtomicLong(0);
private final AtomicLong totalCombinedCoordinatingAndPrimaryBytes = new AtomicLong(0);
private final AtomicLong totalCoordinatingBytes = new AtomicLong(0);
private final AtomicLong totalPrimaryBytes = new AtomicLong(0);
private final AtomicLong totalReplicaBytes = new AtomicLong(0);
protected final AtomicLong totalCombinedCoordinatingAndPrimaryBytes = new AtomicLong(0);
protected final AtomicLong totalCoordinatingBytes = new AtomicLong(0);
protected final AtomicLong totalPrimaryBytes = new AtomicLong(0);
protected final AtomicLong totalReplicaBytes = new AtomicLong(0);
private final AtomicLong coordinatingRejections = new AtomicLong(0);
private final AtomicLong primaryRejections = new AtomicLong(0);
private final AtomicLong replicaRejections = new AtomicLong(0);
protected final AtomicLong coordinatingRejections = new AtomicLong(0);
protected final AtomicLong primaryRejections = new AtomicLong(0);
protected final AtomicLong replicaRejections = new AtomicLong(0);
private final long primaryAndCoordinatingLimits;
private final long replicaLimits;
protected final long primaryAndCoordinatingLimits;
protected final long replicaLimits;
@Inject
public IndexingPressure(Settings settings) {
this.primaryAndCoordinatingLimits = MAX_INDEXING_BYTES.get(settings).getBytes();
this.replicaLimits = (long) (this.primaryAndCoordinatingLimits * 1.5);

View File

@ -0,0 +1,84 @@
/*
* 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.Settings;
import org.opensearch.index.shard.ShardId;
import org.opensearch.index.stats.IndexingPressureStats;
import org.opensearch.index.stats.ShardIndexingPressureStats;
/**
* Sets up classes for node/shard level indexing pressure.
* Provides abstraction and orchestration for indexing pressure interfaces when called from Transport Actions or for Stats.
*/
public class IndexingPressureService {
private final ShardIndexingPressure shardIndexingPressure;
public IndexingPressureService(Settings settings, ClusterService clusterService) {
shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
}
public Releasable markCoordinatingOperationStarted(long bytes, boolean forceExecution) {
if (isShardIndexingPressureEnabled() == false) {
return shardIndexingPressure.markCoordinatingOperationStarted(bytes, forceExecution);
} else {
return () -> {};
}
}
public Releasable markCoordinatingOperationStarted(ShardId shardId, long bytes, boolean forceExecution) {
if (isShardIndexingPressureEnabled()) {
return shardIndexingPressure.markCoordinatingOperationStarted(shardId, bytes, forceExecution);
} else {
return () -> {};
}
}
public Releasable markPrimaryOperationStarted(ShardId shardId, long bytes, boolean forceExecution) {
if (isShardIndexingPressureEnabled()) {
return shardIndexingPressure.markPrimaryOperationStarted(shardId, bytes, forceExecution);
} else {
return shardIndexingPressure.markPrimaryOperationStarted(bytes, forceExecution);
}
}
public Releasable markPrimaryOperationLocalToCoordinatingNodeStarted(ShardId shardId, long bytes) {
if (isShardIndexingPressureEnabled()) {
return shardIndexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(shardId, bytes);
} else {
return shardIndexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(bytes);
}
}
public Releasable markReplicaOperationStarted(ShardId shardId, long bytes, boolean forceExecution) {
if (isShardIndexingPressureEnabled()) {
return shardIndexingPressure.markReplicaOperationStarted(shardId, bytes, forceExecution);
} else {
return shardIndexingPressure.markReplicaOperationStarted(bytes, forceExecution);
}
}
public IndexingPressureStats nodeStats() {
return shardIndexingPressure.stats();
}
public ShardIndexingPressureStats shardStats(CommonStatsFlags statsFlags) {
return shardIndexingPressure.shardStats(statsFlags);
}
private boolean isShardIndexingPressureEnabled() {
return shardIndexingPressure.isShardIndexingPressureEnabled();
}
// visible for testing
ShardIndexingPressure getShardIndexingPressure() {
return shardIndexingPressure;
}
}

View File

@ -0,0 +1,350 @@
/*
* 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.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.ShardIndexingPressureTracker.CommonOperationTracker;
import org.opensearch.index.ShardIndexingPressureTracker.OperationTracker;
import org.opensearch.index.ShardIndexingPressureTracker.PerformanceTracker;
import org.opensearch.index.ShardIndexingPressureTracker.RejectionTracker;
import org.opensearch.index.ShardIndexingPressureTracker.StatsTracker;
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.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* Shard Indexing Pressure is a framework level artefact build on top of IndexingPressure to track incoming indexing request, per shard.
* The interfaces provided by this class will be used by Transport Action layers to start accounting for an incoming request.
* Interfaces returns Releasable which when triggered will release the acquired accounting tokens values and also
* perform necessary actions such as throughput evaluation once the request completes.
* Consumers of these interfaces are expected to trigger close on releasable, reliably for consistency.
*
* Overall ShardIndexingPressure provides:
* 1. Memory Accounting at shard level. This can be enabled/disabled based on dynamic setting.
* 2. Memory Accounting at Node level. Tracking is done using the IndexingPressure artefacts to support feature seamless toggling.
* 3. Interfaces to access the statistics for shard trackers.
*/
public class ShardIndexingPressure extends IndexingPressure {
private static final Logger logger = LogManager.getLogger(ShardIndexingPressure.class);
private final ShardIndexingPressureSettings shardIndexingPressureSettings;
private final ShardIndexingPressureMemoryManager memoryManager;
ShardIndexingPressure(Settings settings, ClusterService clusterService) {
super(settings);
shardIndexingPressureSettings = new ShardIndexingPressureSettings(clusterService, settings, primaryAndCoordinatingLimits);
ClusterSettings clusterSettings = clusterService.getClusterSettings();
this.memoryManager = new ShardIndexingPressureMemoryManager(shardIndexingPressureSettings, clusterSettings, settings);
}
public Releasable markCoordinatingOperationStarted(ShardId shardId, long bytes, boolean forceExecution) {
if(0 == bytes) { return () -> {}; }
long requestStartTime = System.nanoTime();
ShardIndexingPressureTracker tracker = getShardIndexingPressureTracker(shardId);
long nodeCombinedBytes = currentCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes);
long nodeReplicaBytes = currentReplicaBytes.get();
long nodeTotalBytes = nodeCombinedBytes + nodeReplicaBytes;
long shardCombinedBytes = tracker.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(bytes);
boolean shardLevelLimitBreached = false;
if (forceExecution == false) {
boolean nodeLevelLimitBreached = memoryManager.isCoordinatingNodeLimitBreached(tracker, nodeTotalBytes);
if (nodeLevelLimitBreached == false) {
shardLevelLimitBreached = memoryManager.isCoordinatingShardLimitBreached(tracker, nodeTotalBytes, requestStartTime);
}
if (shouldRejectRequest(nodeLevelLimitBreached, shardLevelLimitBreached)) {
coordinatingRejections.getAndIncrement();
currentCombinedCoordinatingAndPrimaryBytes.addAndGet(-bytes);
tracker.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(-bytes);
rejectShardRequest(tracker, bytes, nodeTotalBytes, shardCombinedBytes,
tracker.getCoordinatingOperationTracker().getRejectionTracker(), "coordinating");
}
}
currentCoordinatingBytes.addAndGet(bytes);
totalCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes);
totalCoordinatingBytes.addAndGet(bytes);
StatsTracker statsTracker = tracker.getCoordinatingOperationTracker().getStatsTracker();
statsTracker.incrementCurrentBytes(bytes);
markShardOperationStarted(statsTracker, tracker.getCoordinatingOperationTracker().getPerformanceTracker());
boolean isShadowModeBreach = shardLevelLimitBreached;
return wrapReleasable(() -> {
currentCombinedCoordinatingAndPrimaryBytes.addAndGet(-bytes);
currentCoordinatingBytes.addAndGet(-bytes);
markShardOperationComplete(bytes, requestStartTime, isShadowModeBreach, tracker.getCoordinatingOperationTracker(),
tracker.getCommonOperationTracker());
memoryManager.decreaseShardPrimaryAndCoordinatingLimits(tracker);
tryReleaseTracker(tracker);
});
}
public Releasable markPrimaryOperationLocalToCoordinatingNodeStarted(ShardId shardId, long bytes) {
if(bytes == 0) { return () -> {}; }
ShardIndexingPressureTracker tracker = getShardIndexingPressureTracker(shardId);
currentPrimaryBytes.addAndGet(bytes);
totalPrimaryBytes.addAndGet(bytes);
tracker.getPrimaryOperationTracker().getStatsTracker().incrementCurrentBytes(bytes);
tracker.getPrimaryOperationTracker().getStatsTracker().incrementTotalBytes(bytes);
return wrapReleasable(() -> {
currentPrimaryBytes.addAndGet(-bytes);
tracker.getPrimaryOperationTracker().getStatsTracker().incrementCurrentBytes(-bytes);
});
}
public Releasable markPrimaryOperationStarted(ShardId shardId, long bytes, boolean forceExecution) {
if(0 == bytes) { return () -> {}; }
long requestStartTime = System.nanoTime();
ShardIndexingPressureTracker tracker = getShardIndexingPressureTracker(shardId);
long nodeCombinedBytes = currentCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes);
long nodeReplicaBytes = currentReplicaBytes.get();
long nodeTotalBytes = nodeCombinedBytes + nodeReplicaBytes;
long shardCombinedBytes = tracker.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(bytes);
boolean shardLevelLimitBreached = false;
if (forceExecution == false) {
boolean nodeLevelLimitBreached = memoryManager.isPrimaryNodeLimitBreached(tracker, nodeTotalBytes);
if (nodeLevelLimitBreached == false) {
shardLevelLimitBreached = memoryManager.isPrimaryShardLimitBreached(tracker, nodeTotalBytes, requestStartTime);
}
if (shouldRejectRequest(nodeLevelLimitBreached, shardLevelLimitBreached)) {
primaryRejections.getAndIncrement();
currentCombinedCoordinatingAndPrimaryBytes.addAndGet(-bytes);
tracker.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(-bytes);
rejectShardRequest(tracker, bytes, nodeTotalBytes, shardCombinedBytes,
tracker.getPrimaryOperationTracker().getRejectionTracker(), "primary");
}
}
currentPrimaryBytes.addAndGet(bytes);
totalCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes);
totalPrimaryBytes.addAndGet(bytes);
StatsTracker statsTracker = tracker.getPrimaryOperationTracker().getStatsTracker();
statsTracker.incrementCurrentBytes(bytes);
markShardOperationStarted(statsTracker, tracker.getPrimaryOperationTracker().getPerformanceTracker());
boolean isShadowModeBreach = shardLevelLimitBreached;
return wrapReleasable(() -> {
currentCombinedCoordinatingAndPrimaryBytes.addAndGet(-bytes);
currentPrimaryBytes.addAndGet(-bytes);
markShardOperationComplete(bytes, requestStartTime, isShadowModeBreach, tracker.getPrimaryOperationTracker(),
tracker.getCommonOperationTracker());
memoryManager.decreaseShardPrimaryAndCoordinatingLimits(tracker);
tryReleaseTracker(tracker);
});
}
public Releasable markReplicaOperationStarted(ShardId shardId, long bytes, boolean forceExecution) {
if(0 == bytes) { return () -> {}; }
long requestStartTime = System.nanoTime();
ShardIndexingPressureTracker tracker = getShardIndexingPressureTracker(shardId);
long nodeReplicaBytes = currentReplicaBytes.addAndGet(bytes);
long shardReplicaBytes = tracker.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(bytes);
boolean shardLevelLimitBreached = false;
if (forceExecution == false) {
boolean nodeLevelLimitBreached = memoryManager.isReplicaNodeLimitBreached(tracker, nodeReplicaBytes);
if (nodeLevelLimitBreached == false) {
shardLevelLimitBreached = memoryManager.isReplicaShardLimitBreached(tracker, nodeReplicaBytes, requestStartTime);
}
if (shouldRejectRequest(nodeLevelLimitBreached, shardLevelLimitBreached)) {
replicaRejections.getAndIncrement();
currentReplicaBytes.addAndGet(-bytes);
tracker.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(-bytes);
rejectShardRequest(tracker, bytes, nodeReplicaBytes, shardReplicaBytes,
tracker.getReplicaOperationTracker().getRejectionTracker(), "replica");
}
}
totalReplicaBytes.addAndGet(bytes);
StatsTracker statsTracker = tracker.getReplicaOperationTracker().getStatsTracker();
markShardOperationStarted(statsTracker, tracker.getReplicaOperationTracker().getPerformanceTracker());
boolean isShadowModeBreach = shardLevelLimitBreached;
return wrapReleasable(() -> {
currentReplicaBytes.addAndGet(-bytes);
markShardOperationComplete(bytes, requestStartTime, isShadowModeBreach, tracker.getReplicaOperationTracker());
memoryManager.decreaseShardReplicaLimits(tracker);
tryReleaseTracker(tracker);
});
}
private static Releasable wrapReleasable(Releasable releasable) {
final AtomicBoolean called = new AtomicBoolean();
return () -> {
if (called.compareAndSet(false, true)) {
releasable.close();
} else {
logger.error("ShardIndexingPressure Release is called twice", new IllegalStateException("Releasable is called twice"));
assert false : "ShardIndexingPressure Release is called twice";
}
};
}
private boolean shouldRejectRequest(boolean nodeLevelLimitBreached, boolean shardLevelLimitBreached) {
return nodeLevelLimitBreached ||
(shardLevelLimitBreached && shardIndexingPressureSettings.isShardIndexingPressureEnforced());
}
private void markShardOperationStarted(StatsTracker statsTracker, PerformanceTracker performanceTracker) {
statsTracker.incrementRequestCount();
performanceTracker.incrementTotalOutstandingRequests();
}
private void adjustPerformanceUponCompletion(long bytes, long requestStartTime, StatsTracker statsTracker,
PerformanceTracker performanceTracker) {
long requestEndTime = System.nanoTime();
long requestLatency = TimeUnit.NANOSECONDS.toMillis(requestEndTime - requestStartTime);
performanceTracker.addLatencyInMillis(requestLatency);
performanceTracker.updateLastSuccessfulRequestTimestamp(requestEndTime);
performanceTracker.resetTotalOutstandingRequests();
if(requestLatency > 0) {
calculateRequestThroughput(bytes, requestLatency, performanceTracker, statsTracker);
}
}
private void calculateRequestThroughput(long bytes, long requestLatency, PerformanceTracker performanceTracker,
StatsTracker statsTracker) {
double requestThroughput = (double) bytes / requestLatency;
performanceTracker.addNewThroughout(requestThroughput);
if (performanceTracker.getThroughputMovingQueueSize() > shardIndexingPressureSettings.getRequestSizeWindow()) {
double front = performanceTracker.getFirstThroughput();
double movingAverage = memoryManager.calculateMovingAverage(performanceTracker.getThroughputMovingAverage(), front,
requestThroughput, shardIndexingPressureSettings.getRequestSizeWindow());
performanceTracker.updateThroughputMovingAverage(Double.doubleToLongBits(movingAverage));
} else {
double movingAverage = (double) statsTracker.getTotalBytes() / performanceTracker.getLatencyInMillis();
performanceTracker.updateThroughputMovingAverage(Double.doubleToLongBits(movingAverage));
}
}
private void markShardOperationComplete(long bytes, long requestStartTime, boolean isShadowModeBreach,
OperationTracker operationTracker, CommonOperationTracker commonOperationTracker) {
commonOperationTracker.incrementCurrentCombinedCoordinatingAndPrimaryBytes(-bytes);
commonOperationTracker.incrementTotalCombinedCoordinatingAndPrimaryBytes(bytes);
markShardOperationComplete(bytes, requestStartTime, isShadowModeBreach, operationTracker);
}
private void markShardOperationComplete(long bytes, long requestStartTime, boolean isShadowModeBreach,
OperationTracker operationTracker) {
StatsTracker statsTracker = operationTracker.getStatsTracker();
statsTracker.incrementCurrentBytes(-bytes);
statsTracker.incrementTotalBytes(bytes);
// In shadow mode if request was intended to be rejected, we do not account it for dynamic rejection parameters
if (isShadowModeBreach == false) {
adjustPerformanceUponCompletion(bytes, requestStartTime, statsTracker, operationTracker.getPerformanceTracker());
}
}
private void tryReleaseTracker(ShardIndexingPressureTracker tracker) {
memoryManager.tryTrackerCleanupFromHotStore(tracker,
() -> (tracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes() == 0 &&
tracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes() == 0));
}
private void rejectShardRequest(ShardIndexingPressureTracker tracker, long bytes, long nodeTotalBytes, long shardTotalBytes,
RejectionTracker rejectionTracker, String operationType) {
long nodeBytesWithoutOperation = nodeTotalBytes - bytes;
long shardBytesWithoutOperation = shardTotalBytes - bytes;
ShardId shardId = tracker.getShardId();
rejectionTracker.incrementTotalRejections();
throw new OpenSearchRejectedExecutionException("rejected execution of " + operationType + " operation [" +
"shard_detail=[" + shardId.getIndexName() + "][" + shardId.id() + "], " +
"shard_total_bytes=" + shardBytesWithoutOperation + ", " +
"shard_operation_bytes=" + bytes + ", " +
"shard_max_coordinating_and_primary_bytes=" + tracker.getPrimaryAndCoordinatingLimits() + ", " +
"shard_max_replica_bytes=" + tracker.getReplicaLimits() + "] OR [" +
"node_total_bytes=" + nodeBytesWithoutOperation + ", " +
"node_operation_bytes=" + bytes + ", " +
"node_max_coordinating_and_primary_bytes=" + primaryAndCoordinatingLimits + ", " +
"node_max_replica_bytes=" + replicaLimits + "]", false);
}
public ShardIndexingPressureStats shardStats(CommonStatsFlags statsFlags) {
if (statsFlags.includeOnlyTopIndexingPressureMetrics()) {
return topStats();
} else {
ShardIndexingPressureStats allStats = shardStats();
if (statsFlags.includeAllShardIndexingPressureTrackers()) {
allStats.addAll(coldStats());
}
return allStats;
}
}
ShardIndexingPressureStats shardStats() {
Map<ShardId, IndexingPressurePerShardStats> statsPerShard = new HashMap<>();
boolean isEnforcedMode = shardIndexingPressureSettings.isShardIndexingPressureEnforced();
for (Map.Entry<ShardId, ShardIndexingPressureTracker> shardEntry :
memoryManager.getShardIndexingPressureHotStore().entrySet()) {
IndexingPressurePerShardStats shardStats = new IndexingPressurePerShardStats(shardEntry.getValue(), isEnforcedMode);
statsPerShard.put(shardEntry.getKey(), shardStats);
}
return new ShardIndexingPressureStats(statsPerShard, memoryManager.getTotalNodeLimitsBreachedRejections(),
memoryManager.getTotalLastSuccessfulRequestLimitsBreachedRejections(),
memoryManager.getTotalThroughputDegradationLimitsBreachedRejections(),
shardIndexingPressureSettings.isShardIndexingPressureEnabled(), isEnforcedMode);
}
ShardIndexingPressureStats coldStats() {
Map<ShardId, IndexingPressurePerShardStats> statsPerShard = new HashMap<>();
boolean isEnforcedMode = shardIndexingPressureSettings.isShardIndexingPressureEnforced();
for (Map.Entry<ShardId, ShardIndexingPressureTracker> shardEntry :
memoryManager.getShardIndexingPressureColdStore().entrySet()) {
IndexingPressurePerShardStats shardStats = new IndexingPressurePerShardStats(shardEntry.getValue(), isEnforcedMode);
statsPerShard.put(shardEntry.getKey(), shardStats);
}
return new ShardIndexingPressureStats(statsPerShard, memoryManager.getTotalNodeLimitsBreachedRejections(),
memoryManager.getTotalLastSuccessfulRequestLimitsBreachedRejections(),
memoryManager.getTotalThroughputDegradationLimitsBreachedRejections(),
shardIndexingPressureSettings.isShardIndexingPressureEnabled(), isEnforcedMode);
}
ShardIndexingPressureStats topStats() {
return new ShardIndexingPressureStats(Collections.emptyMap(), memoryManager.getTotalNodeLimitsBreachedRejections(),
memoryManager.getTotalLastSuccessfulRequestLimitsBreachedRejections(),
memoryManager.getTotalThroughputDegradationLimitsBreachedRejections(),
shardIndexingPressureSettings.isShardIndexingPressureEnabled(),
shardIndexingPressureSettings.isShardIndexingPressureEnforced());
}
ShardIndexingPressureTracker getShardIndexingPressureTracker(ShardId shardId) {
return memoryManager.getShardIndexingPressureTracker(shardId);
}
public boolean isShardIndexingPressureEnabled() {
return shardIndexingPressureSettings.isShardIndexingPressureEnabled();
}
}

View File

@ -0,0 +1,477 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.index.ShardIndexingPressureTracker.OperationTracker;
import org.opensearch.index.ShardIndexingPressureTracker.PerformanceTracker;
import org.opensearch.index.ShardIndexingPressureTracker.RejectionTracker;
import org.opensearch.index.ShardIndexingPressureTracker.StatsTracker;
import org.opensearch.index.shard.ShardId;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.BiPredicate;
import java.util.function.BooleanSupplier;
import java.util.function.LongSupplier;
import java.util.function.ToLongFunction;
/**
* The Shard Indexing Pressure Memory Manager is the construct responsible for increasing and decreasing the allocated shard limit
* based on incoming requests. A shard limits defines the maximum memory that a shard can occupy in the heap for request objects.
*
* Based on the overall memory utilization on the node, and current traffic needs shard limits will be modified:
*
* 1. If the limits assigned to a shard is breached (Primary Parameter) while the node level overall occupancy across all shards
* is not greater than primary_parameter.node.soft_limit, MemoryManager will increase the shard limits without any deeper evaluation.
* 2. If the limits assigned to the shard is breached(Primary Parameter) and the node level overall occupancy across all shards
* is greater than primary_parameter.node.soft_limit, then MemoryManager will evaluate deeper parameters for shards to identify any
* issues, such as throughput degradation (Secondary Parameter - 1) and time since last request was successful (Secondary Parameter - 2).
* This helps identify detect any duress state with the shard, requesting more memory.
*
* Secondary Parameters covered above:
* 1. ThroughputDegradationLimitsBreached - When the moving window throughput average has increased by a factor compared to
* the historical throughput average. If the factor by which it has increased is greater than the degradation limit threshold, this
* parameter is considered to be breached.
* 2. LastSuccessfulRequestDurationLimitsBreached - When the time since the last successful request completed is greater than the max
* timeout threshold value, while there a number of outstanding requests greater than the max outstanding requests then this parameter
* is considered to be breached.
*
* MemoryManager attempts to increase of decrease the shard limits in case the shard utilization goes below operating_factor.lower or
* goes above operating_factor.upper of current shard limits. MemoryManager attempts to update the new shard limit such that the new value
* remains withing the operating_factor.optimal range of current shard utilization.
*
*/
public class ShardIndexingPressureMemoryManager {
private static final Logger logger = LogManager.getLogger(ShardIndexingPressureMemoryManager.class);
/**
* Shard operating factor can be evaluated using currentShardBytes/shardLimits. Outcome of this expression is categorized as
* lower, optimal and upper boundary, and appropriate action is taken once the below defined threshold values are breached.
*/
public static final Setting<Double> LOWER_OPERATING_FACTOR =
Setting.doubleSetting("shard_indexing_pressure.operating_factor.lower", 0.75d, 0.0d,
Setting.Property.NodeScope, Setting.Property.Dynamic);
public static final Setting<Double> OPTIMAL_OPERATING_FACTOR =
Setting.doubleSetting("shard_indexing_pressure.operating_factor.optimal", 0.85d, 0.0d,
Setting.Property.NodeScope, Setting.Property.Dynamic);
public static final Setting<Double> UPPER_OPERATING_FACTOR =
Setting.doubleSetting("shard_indexing_pressure.operating_factor.upper", 0.95d, 0.0d,
Setting.Property.NodeScope, Setting.Property.Dynamic);
/**
* This determines the max time elapsed since any request was processed successfully. Appropriate action is taken
* once the below below defined threshold value is breached.
*/
public static final Setting<TimeValue> SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT =
Setting.positiveTimeSetting("shard_indexing_pressure.secondary_parameter.successful_request.elapsed_timeout",
TimeValue.timeValueMillis(300000), Setting.Property.NodeScope, Setting.Property.Dynamic);
/**
* This determines the max outstanding request that are yet to be processed successfully. Appropriate
* action is taken once the below defined threshold value is breached.
*/
public static final Setting<Integer> MAX_OUTSTANDING_REQUESTS =
Setting.intSetting("shard_indexing_pressure.secondary_parameter.successful_request.max_outstanding_requests",
100, Setting.Property.NodeScope, Setting.Property.Dynamic);
/**
* Degradation for a shard can be evaluated using average throughput of last N requests,
* where N being {@link ShardIndexingPressureSettings#REQUEST_SIZE_WINDOW}, divided by lifetime average throughput.
* Appropriate action is taken once the outcome of above expression breaches the below defined threshold value is breached.
*/
public static final Setting<Double> THROUGHPUT_DEGRADATION_LIMITS =
Setting.doubleSetting("shard_indexing_pressure.secondary_parameter.throughput.degradation_factor", 5.0d, 1.0d,
Setting.Property.NodeScope, Setting.Property.Dynamic);
/**
* The node level soft limit determines when the secondary parameters for shard is to be evaluated for degradation.
*/
public static final Setting<Double> NODE_SOFT_LIMIT =
Setting.doubleSetting("shard_indexing_pressure.primary_parameter.node.soft_limit", 0.7d, 0.0d,
Setting.Property.NodeScope, Setting.Property.Dynamic);
private final AtomicLong totalNodeLimitsBreachedRejections = new AtomicLong();
private final AtomicLong totalLastSuccessfulRequestLimitsBreachedRejections = new AtomicLong();
private final AtomicLong totalThroughputDegradationLimitsBreachedRejections = new AtomicLong();
private final ShardIndexingPressureSettings shardIndexingPressureSettings;
private final ShardIndexingPressureStore shardIndexingPressureStore;
private volatile double lowerOperatingFactor;
private volatile double optimalOperatingFactor;
private volatile double upperOperatingFactor;
private volatile TimeValue 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.shardIndexingPressureStore = new ShardIndexingPressureStore(shardIndexingPressureSettings, clusterSettings, settings);
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);
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);
}
/**
* Checks if the node level memory threshold is breached for coordinating operations.
*/
boolean isCoordinatingNodeLimitBreached(ShardIndexingPressureTracker tracker, long nodeTotalBytes) {
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());
incrementNodeLimitBreachedRejectionCount(tracker.getCoordinatingOperationTracker().getRejectionTracker());
return true;
}
return false;
}
/**
* Checks if the shard level memory threshold is breached for coordinating operations.
*/
boolean isCoordinatingShardLimitBreached(ShardIndexingPressureTracker tracker, long nodeTotalBytes, long requestStartTime) {
// Shard memory limits is breached when the current utilization is greater than operating_factor.upper limit.
long shardCombinedBytes = tracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes();
long shardPrimaryAndCoordinatingLimits = tracker.getPrimaryAndCoordinatingLimits();
boolean shardMemoryLimitsBreached = ((double)shardCombinedBytes / shardPrimaryAndCoordinatingLimits) > this.upperOperatingFactor;
if(shardMemoryLimitsBreached) {
BooleanSupplier increaseShardLimitSupplier = () -> increaseShardLimits(tracker.getShardId(),
this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits(),
() -> tracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes(),
tracker::getPrimaryAndCoordinatingLimits,
ShardIndexingPressureTracker::getPrimaryAndCoordinatingLimits,
tracker::compareAndSetPrimaryAndCoordinatingLimits);
return onShardLimitBreached(nodeTotalBytes, this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits(),
requestStartTime, tracker.getCoordinatingOperationTracker(), increaseShardLimitSupplier);
} else {
return false;
}
}
/**
* Checks if the node level memory threshold is breached for primary operations.
*/
boolean isPrimaryNodeLimitBreached(ShardIndexingPressureTracker tracker, long nodeTotalBytes) {
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());
incrementNodeLimitBreachedRejectionCount(tracker.getPrimaryOperationTracker().getRejectionTracker());
return true;
}
return false;
}
/**
* Checks if the shard level memory threshold is breached for primary operations.
*/
boolean isPrimaryShardLimitBreached(ShardIndexingPressureTracker tracker, long nodeTotalBytes, long requestStartTime) {
// Shard memory limits is breached when the current utilization is greater than operating_factor.upper limit.
long shardCombinedBytes = tracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes();
long shardPrimaryAndCoordinatingLimits = tracker.getPrimaryAndCoordinatingLimits();
boolean shardMemoryLimitsBreached = ((double)shardCombinedBytes / shardPrimaryAndCoordinatingLimits) > this.upperOperatingFactor;
if(shardMemoryLimitsBreached) {
BooleanSupplier increaseShardLimitSupplier = () -> increaseShardLimits(tracker.getShardId(),
this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits(),
() -> tracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes(),
tracker::getPrimaryAndCoordinatingLimits,
ShardIndexingPressureTracker::getPrimaryAndCoordinatingLimits,
tracker::compareAndSetPrimaryAndCoordinatingLimits);
return onShardLimitBreached(nodeTotalBytes, this.shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits(),
requestStartTime, tracker.getPrimaryOperationTracker(), increaseShardLimitSupplier);
} else {
return false;
}
}
/**
* Checks if the node level memory threshold is breached for replica operations.
*/
boolean isReplicaNodeLimitBreached(ShardIndexingPressureTracker tracker, long nodeReplicaBytes) {
if(nodeReplicaBytes > this.shardIndexingPressureSettings.getNodeReplicaLimits()) {
logger.debug("Node limits breached for replica operation [node_replica_bytes={} , " +
"node_replica_limits={}]", nodeReplicaBytes, this.shardIndexingPressureSettings.getNodeReplicaLimits());
incrementNodeLimitBreachedRejectionCount(tracker.getReplicaOperationTracker().getRejectionTracker());
return true;
}
return false;
}
/**
* Checks if the shard level memory threshold is breached for replica operations.
*/
boolean isReplicaShardLimitBreached(ShardIndexingPressureTracker tracker, long nodeReplicaBytes, long requestStartTime) {
// Shard memory limits is breached when the current utilization is greater than operating_factor.upper limit.
long shardReplicaBytes = tracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes();
long shardReplicaLimits = tracker.getReplicaLimits();
final boolean shardMemoryLimitsBreached = ((double)shardReplicaBytes / shardReplicaLimits) > this.upperOperatingFactor;
if(shardMemoryLimitsBreached) {
BooleanSupplier increaseShardLimitSupplier = () -> increaseShardLimits(tracker.getShardId(),
this.shardIndexingPressureSettings.getNodeReplicaLimits(),
() -> tracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes(),
tracker::getReplicaLimits,
ShardIndexingPressureTracker::getReplicaLimits,
tracker::compareAndSetReplicaLimits);
return onShardLimitBreached(nodeReplicaBytes, this.shardIndexingPressureSettings.getNodeReplicaLimits(),
requestStartTime, tracker.getReplicaOperationTracker(), increaseShardLimitSupplier);
} else {
return false;
}
}
void decreaseShardPrimaryAndCoordinatingLimits(ShardIndexingPressureTracker tracker) {
decreaseShardLimits(tracker.getShardId(),
() -> tracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes(),
tracker::getPrimaryAndCoordinatingLimits,
tracker::compareAndSetPrimaryAndCoordinatingLimits,
shardIndexingPressureSettings.getShardPrimaryAndCoordinatingBaseLimits());
}
void decreaseShardReplicaLimits(ShardIndexingPressureTracker tracker) {
decreaseShardLimits(tracker.getShardId(),
() -> tracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes(),
tracker::getReplicaLimits,
tracker::compareAndSetReplicaLimits,
shardIndexingPressureSettings.getShardReplicaBaseLimits());
}
ShardIndexingPressureTracker getShardIndexingPressureTracker(ShardId shardId) {
return shardIndexingPressureStore.getShardIndexingPressureTracker(shardId);
}
Map<ShardId, ShardIndexingPressureTracker> getShardIndexingPressureHotStore() {
return shardIndexingPressureStore.getShardIndexingPressureHotStore();
}
Map<ShardId, ShardIndexingPressureTracker> getShardIndexingPressureColdStore() {
return shardIndexingPressureStore.getShardIndexingPressureColdStore();
}
void tryTrackerCleanupFromHotStore(ShardIndexingPressureTracker tracker, BooleanSupplier condition) {
shardIndexingPressureStore.tryTrackerCleanupFromHotStore(tracker, condition);
}
double calculateMovingAverage(long currentAverage, double frontValue, double currentValue, int count) {
if(count > 0) {
return ((Double.longBitsToDouble(currentAverage) * count) + currentValue - frontValue) / count;
} else {
return currentValue;
}
}
long getTotalNodeLimitsBreachedRejections() {
return totalNodeLimitsBreachedRejections.get();
}
long getTotalLastSuccessfulRequestLimitsBreachedRejections() {
return totalLastSuccessfulRequestLimitsBreachedRejections.get();
}
long getTotalThroughputDegradationLimitsBreachedRejections() {
return totalThroughputDegradationLimitsBreachedRejections.get();
}
/**
* Verifies and returns true if the shard limit is hard-breached i.e. shard limit cannot be increased further. Otherwise
* increases the shard limit and returns false.
*/
private boolean onShardLimitBreached(long nodeTotalBytes, long nodeLimit, long requestStartTime, OperationTracker operationTracker,
BooleanSupplier increaseShardLimitSupplier) {
// Secondary Parameters (i.e. LastSuccessfulRequestDuration and Throughput) is taken into consideration when
// the current node utilization is greater than primary_parameter.node.soft_limit of total node limits.
if(((double)nodeTotalBytes / nodeLimit) < this.nodeSoftLimit) {
boolean isShardLimitsIncreased = increaseShardLimitSupplier.getAsBoolean();
if (isShardLimitsIncreased == false) {
incrementNodeLimitBreachedRejectionCount(operationTracker.getRejectionTracker());
}
return !isShardLimitsIncreased;
} else {
boolean shardLastSuccessfulRequestDurationLimitsBreached =
evaluateLastSuccessfulRequestDurationLimitsBreached(operationTracker.getPerformanceTracker(), requestStartTime);
if (shardLastSuccessfulRequestDurationLimitsBreached) {
operationTracker.getRejectionTracker().incrementLastSuccessfulRequestLimitsBreachedRejections();
this.totalLastSuccessfulRequestLimitsBreachedRejections.incrementAndGet();
return true;
}
boolean shardThroughputDegradationLimitsBreached =
evaluateThroughputDegradationLimitsBreached(operationTracker.getPerformanceTracker(),
operationTracker.getStatsTracker(), primaryAndCoordinatingThroughputDegradationLimits);
if (shardThroughputDegradationLimitsBreached) {
operationTracker.getRejectionTracker().incrementThroughputDegradationLimitsBreachedRejections();
this.totalThroughputDegradationLimitsBreachedRejections.incrementAndGet();
return true;
}
boolean isShardLimitsIncreased = increaseShardLimitSupplier.getAsBoolean();
if (isShardLimitsIncreased == false) {
incrementNodeLimitBreachedRejectionCount(operationTracker.getRejectionTracker());
}
return !isShardLimitsIncreased;
}
}
private boolean increaseShardLimits(ShardId shardId, long nodeLimit,
LongSupplier shardCurrentBytesSupplier, LongSupplier shardLimitSupplier,
ToLongFunction<ShardIndexingPressureTracker> getShardLimitFunction,
BiPredicate<Long, Long> updateShardLimitPredicate) {
long currentShardLimit;
long newShardLimit;
do {
currentShardLimit = shardLimitSupplier.getAsLong();
long shardCurrentBytes = shardCurrentBytesSupplier.getAsLong();
if(((double)shardCurrentBytes / currentShardLimit) > this.upperOperatingFactor) {
newShardLimit = (long)(shardCurrentBytes / this.optimalOperatingFactor);
long totalShardLimitsExceptCurrentShard = this.shardIndexingPressureStore.getShardIndexingPressureHotStore()
.entrySet().stream()
.filter(entry -> (shardId != entry.getKey()))
.map(Map.Entry::getValue)
.mapToLong(getShardLimitFunction).sum();
if (totalShardLimitsExceptCurrentShard + newShardLimit > nodeLimit) {
logger.debug("Failed To Increase Shard Limit [shard_detail=[{}][{}}], " +
"shard_current_limit_bytes={}, " + "total_shard_limits_bytes_except_current_shard={}, " +
"expected_shard_limits_bytes={}]",
shardId.getIndexName(), shardId.id(), currentShardLimit, totalShardLimitsExceptCurrentShard, newShardLimit);
return false;
}
} else {
return true;
}
} while(!updateShardLimitPredicate.test(currentShardLimit, newShardLimit));
logger.debug("Increased Shard Limit [" +
"shard_detail=[{}][{}], old_shard_limit_bytes={}, " + "new_shard_limit_bytes={}]",
shardId.getIndexName(), shardId.id(), currentShardLimit, newShardLimit);
return true;
}
private void decreaseShardLimits(ShardId shardId, LongSupplier shardCurrentBytesSupplier, LongSupplier shardLimitSupplier,
BiPredicate<Long, Long> updateShardLimitPredicate, long shardBaseLimit) {
long currentShardLimit;
long newShardLimit;
do {
currentShardLimit = shardLimitSupplier.getAsLong();
long shardCurrentBytes = shardCurrentBytesSupplier.getAsLong();
newShardLimit = Math.max((long) (shardCurrentBytes / this.optimalOperatingFactor), shardBaseLimit);
if (((double)shardCurrentBytes / currentShardLimit) > this.lowerOperatingFactor) {
logger.debug("Shard Limits Already Decreased [" +
"shard_detail=[{}][{}], " + "current_shard_limit_bytes={}, " +
"expected_shard_limit_bytes={}]",
shardId.getIndexName(), shardId.id(), currentShardLimit, newShardLimit);
return;
}
} while(!updateShardLimitPredicate.test(currentShardLimit,newShardLimit));
logger.debug("Decreased Shard Limit [shard_detail=[{}][{}], " +
"old_shard_limit_bytes={}, new_shard_limit_bytes={}]",
shardId.getIndexName(), shardId.id(), currentShardLimit, newShardLimit);
}
/**
* This evaluation returns true if throughput of last N request divided by the total lifetime requests throughput is greater than
* the degradation limits threshold.
*/
private boolean evaluateThroughputDegradationLimitsBreached(PerformanceTracker performanceTracker, StatsTracker statsTracker,
double degradationLimits) {
double throughputMovingAverage = Double.longBitsToDouble(performanceTracker.getThroughputMovingAverage());
long throughputMovingQueueSize = performanceTracker.getThroughputMovingQueueSize();
double throughputHistoricalAverage = (double)statsTracker.getTotalBytes() / performanceTracker.getLatencyInMillis();
return throughputMovingAverage > 0 && throughputMovingQueueSize >= this.shardIndexingPressureSettings.getRequestSizeWindow() &&
throughputHistoricalAverage / throughputMovingAverage > degradationLimits;
}
/**
* This evaluation returns true if the difference in the current timestamp and last successful request timestamp is greater than
* the successful request elapsed-timeout threshold, and the total number of outstanding requests is greater than
* the maximum outstanding request-count threshold.
*/
private boolean evaluateLastSuccessfulRequestDurationLimitsBreached(PerformanceTracker performanceTracker, long requestStartTime) {
return (performanceTracker.getLastSuccessfulRequestTimestamp() > 0) &&
(requestStartTime - performanceTracker.getLastSuccessfulRequestTimestamp()) > this.successfulRequestElapsedTimeout.nanos() &&
performanceTracker.getTotalOutstandingRequests() > 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(TimeValue 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;
}
private void incrementNodeLimitBreachedRejectionCount(RejectionTracker rejectionTracker) {
rejectionTracker.incrementNodeLimitsBreachedRejections();
this.totalNodeLimitsBreachedRejections.incrementAndGet();
}
}

View File

@ -0,0 +1,149 @@
/*
* Copyright OpenSearch Contributors.
* SPDX-License-Identifier: Apache-2.0
*/
package org.opensearch.index;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import java.util.Iterator;
import java.util.Objects;
/**
* This class contains all the settings which are required and owned by {TODO link ShardIndexingPressure}. These will be
* referenced/used in ShardIndexingPressure, as well as its dependent components, i.e.
* {TODO link ShardIndexingPressureMemoryManager} and {TODO link ShardIndexingPressureStore}
*/
public final class ShardIndexingPressureSettings {
public static final String SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY = "shard_indexing_pressure_enabled";
public static final Setting<Boolean> SHARD_INDEXING_PRESSURE_ENABLED =
Setting.boolSetting("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<Boolean> SHARD_INDEXING_PRESSURE_ENFORCED =
Setting.boolSetting("shard_indexing_pressure.enforced", false, Setting.Property.Dynamic, Setting.Property.NodeScope);
/**
* This represents the window size of last N request sampled and considered for throughput evaluation.
*/
public static final Setting<Integer> REQUEST_SIZE_WINDOW =
Setting.intSetting("shard_indexing_pressure.secondary_parameter.throughput.request_size_window", 2000,
Setting.Property.NodeScope, Setting.Property.Dynamic);
/**
* This represents the base limit set for the utilization of every shard. Will be initilized as 1/1000th bytes of node limits.
*/
public static final Setting<Double> SHARD_MIN_LIMIT =
Setting.doubleSetting("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;
private static ClusterService clusterService;
public ShardIndexingPressureSettings(ClusterService clusterService, Settings settings, long primaryAndCoordinatingLimits) {
ShardIndexingPressureSettings.clusterService = clusterService;
ClusterSettings clusterSettings = clusterService.getClusterSettings();
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);
}
public static boolean isShardIndexingPressureAttributeEnabled() {
// Null check is required only for bwc tests which start node without initializing cluster service.
// In actual run time, clusterService will never be null.
if (Objects.nonNull(clusterService) && clusterService.getClusterApplierService().isInitialClusterStateSet()) {
Iterator<DiscoveryNode> nodes = clusterService.state().getNodes().getNodes().valuesIt();
while (nodes.hasNext()) {
if (Boolean.parseBoolean(nodes.next().getAttributes().get(SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY)) == false) {
return false;
}
}
}
return true;
}
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);
}
}

View File

@ -0,0 +1,125 @@
/*
* 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.HashMap;
import java.util.Map;
import java.util.function.BooleanSupplier;
import static java.util.Objects.isNull;
/**
* Shard indexing pressure store acts as a central repository for all the shard-level tracker objects currently being
* used at the Node level, for tracking indexing pressure requests.
* Store manages the tracker lifecycle, from creation, access, until it is evicted to be collected.
*
* Trackers are maintained at two levels for access simplicity and better memory management:
*
* 1. shardIndexingPressureHotStore : As the name suggests, it is hot store for tracker objects which are currently live i.e. being used
* to track an ongoing request.
*
* 2. shardIndexingPressureColdStore : This acts as the store for all the shard tracking objects which are currently being used
* by the framework. In addition to hot trackers, the recently used trackers which are although not currently live, but again can be used
* in near future, are also part of this store. To limit any memory implications, this store has an upper limit on the maximum number of
* trackers its can hold at any given time, which is a configurable dynamic setting.
*
* 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 purge when upper limit is hit.
* 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 and registered in both the stores for concurrent accesses.
*
* Note: The implementation of shardIndexingPressureColdStore methods is such that get,
* update and evict operations can be abstracted out to support any other strategy such as LRU, if
* discovered a need later.
*
*/
public class ShardIndexingPressureStore {
// This represents the maximum value for the cold store size.
public static final Setting<Integer> MAX_COLD_STORE_SIZE =
Setting.intSetting("shard_indexing_pressure.cache_store.max_size", 200, 100, 1000,
Setting.Property.NodeScope, Setting.Property.Dynamic);
private final Map<ShardId, ShardIndexingPressureTracker> shardIndexingPressureHotStore =
ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency();
private final Map<ShardId, ShardIndexingPressureTracker> shardIndexingPressureColdStore = new HashMap<>();
private final ShardIndexingPressureSettings shardIndexingPressureSettings;
private volatile int maxColdStoreSize;
public ShardIndexingPressureStore(ShardIndexingPressureSettings shardIndexingPressureSettings,
ClusterSettings clusterSettings, Settings settings) {
this.shardIndexingPressureSettings = shardIndexingPressureSettings;
this.maxColdStoreSize = MAX_COLD_STORE_SIZE.get(settings).intValue();
clusterSettings.addSettingsUpdateConsumer(MAX_COLD_STORE_SIZE, this::setMaxColdStoreSize);
}
public ShardIndexingPressureTracker getShardIndexingPressureTracker(ShardId shardId) {
ShardIndexingPressureTracker tracker = shardIndexingPressureHotStore.get(shardId);
if (isNull(tracker)) {
// Attempt from Indexing pressure cold store
tracker = shardIndexingPressureColdStore.get(shardId);
// If not already present in cold store instantiate a new one
if (isNull(tracker)) {
tracker = shardIndexingPressureHotStore.computeIfAbsent(shardId, (k) ->
new ShardIndexingPressureTracker(shardId,
this.shardIndexingPressureSettings.getShardPrimaryAndCoordinatingBaseLimits(),
this.shardIndexingPressureSettings.getShardReplicaBaseLimits())
);
// Write through into the cold store for future reference
updateShardIndexingPressureColdStore(tracker);
} else {
// Attempt update tracker to the hot store and return the tracker which finally made to the hot store to avoid any race
ShardIndexingPressureTracker newTracker = shardIndexingPressureHotStore.putIfAbsent(shardId, tracker);
tracker = newTracker == null ? tracker : newTracker;
}
}
return tracker;
}
public Map<ShardId, ShardIndexingPressureTracker> getShardIndexingPressureHotStore() {
return Collections.unmodifiableMap(shardIndexingPressureHotStore);
}
public Map<ShardId, ShardIndexingPressureTracker> getShardIndexingPressureColdStore() {
return Collections.unmodifiableMap(shardIndexingPressureColdStore);
}
public void tryTrackerCleanupFromHotStore(ShardIndexingPressureTracker tracker, BooleanSupplier condition) {
if (condition.getAsBoolean()) {
// Try inserting into cold store again in case there was an eviction triggered
shardIndexingPressureColdStore.putIfAbsent(tracker.getShardId(), tracker);
// Remove from the hot store
shardIndexingPressureHotStore.remove(tracker.getShardId(), tracker);
}
}
/**
* This is used to update the reference of tracker in cold store, to be re-used later of tracker is removed from hot store upon request
* completion. When the cold store size reaches maximum, all the tracker objects in cold store are flushed. Flush is a less frequent
* (periodic) operation, can be sized based on workload. It is okay to not to synchronize counters being flushed, as
* objects in the cold store are only empty references, and can be re-initialized if needed.
*/
private void updateShardIndexingPressureColdStore(ShardIndexingPressureTracker tracker) {
if (shardIndexingPressureColdStore.size() > maxColdStoreSize) {
shardIndexingPressureColdStore.clear();
}
shardIndexingPressureColdStore.put(tracker.getShardId(), tracker);
}
private void setMaxColdStoreSize(int maxColdStoreSize) {
this.maxColdStoreSize = maxColdStoreSize;
}
}

View File

@ -0,0 +1,295 @@
/*
* 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 is responsible for all the tracking that needs to be performed at every Shard Level for Indexing Operations on the node.
* Info is maintained at the granularity of three kinds of write operation (tasks) on the node i.e. coordinating, primary and replica.
* This is useful in evaluating the shard indexing back-pressure on the node, to throttle requests and also to publish runtime stats.
*
* There can be four kinds of operation tracking on a node which needs to performed for a shard:
* 1. Coordinating Operation : To track all the individual shard bulk request on the coordinator node.
* 2. Primary Operation : To track all the individual shard bulk request on the primary node.
* 3. Replica Operation : To track all the individual shard bulk request on the replica node.
* 4. Common Operation : To track values applicable across the specific shard role.
*
* ShardIndexingPressureTracker therefore provides the construct to track all the write requests targeted for a ShardId on the node,
* across all possible transport-write-actions i.e. Coordinator, Primary and Replica.
* Tracker is uniquely identified against a Shard-Id on the node. Currently the knowledge of shard roles (such as primary vs replica)
* is not explicit to the tracker, and it is able to track different values simultaneously based on the interaction hooks of the
* operation type i.e. write-action layers.
*
* There is room for introducing more unique identity to the trackers based on Shard-Role or Shard-Allocation-Id, but that will also
* increase the complexity of handling shard-lister events and handling other race scenarios such as request-draining etc.
* To prefer simplicity we have modelled by keeping explicit fields for different operation tracking, while tracker by itself is
* agnostic of the actual shard role.
*/
public class ShardIndexingPressureTracker {
private final ShardId shardId;
private final AtomicLong primaryAndCoordinatingLimits;
private final AtomicLong replicaLimits;
private final OperationTracker coordinatingOperationTracker = new OperationTracker();
private final OperationTracker primaryOperationTracker = new OperationTracker();
private final OperationTracker replicaOperationTracker = new OperationTracker();
private final CommonOperationTracker commonOperationTracker = new CommonOperationTracker();
public ShardIndexingPressureTracker(ShardId shardId, long initialPrimaryAndCoordinatingLimits, long initialReplicaLimits) {
this.shardId = shardId;
this.primaryAndCoordinatingLimits = new AtomicLong(initialPrimaryAndCoordinatingLimits);
this.replicaLimits = new AtomicLong(initialReplicaLimits);
}
public ShardId getShardId() {
return shardId;
}
public long getPrimaryAndCoordinatingLimits() {
return primaryAndCoordinatingLimits.get();
}
public boolean compareAndSetPrimaryAndCoordinatingLimits(long expectedValue, long newValue) {
return primaryAndCoordinatingLimits.compareAndSet(expectedValue, newValue);
}
public long getReplicaLimits() {
return replicaLimits.get();
}
public boolean compareAndSetReplicaLimits(long expectedValue, long newValue) {
return replicaLimits.compareAndSet(expectedValue, newValue);
}
public OperationTracker getCoordinatingOperationTracker() {
return coordinatingOperationTracker;
}
public OperationTracker getPrimaryOperationTracker() {
return primaryOperationTracker;
}
public OperationTracker getReplicaOperationTracker() {
return replicaOperationTracker;
}
public CommonOperationTracker getCommonOperationTracker() {
return commonOperationTracker;
}
/**
* OperationTracker bundles the different kind of attributes which needs to be tracked for every operation, per shard:
* a. StatsTracker : To track request level aggregated statistics for a shard
* b. RejectionTracker : To track the rejection statistics for a shard
* c. Performance Tracker : To track the request performance statistics for a shard
*/
public static class OperationTracker {
private final StatsTracker statsTracker = new StatsTracker();
private final RejectionTracker rejectionTracker = new RejectionTracker();
private final PerformanceTracker performanceTracker = new PerformanceTracker();
public StatsTracker getStatsTracker() {
return statsTracker;
}
public RejectionTracker getRejectionTracker() {
return rejectionTracker;
}
public PerformanceTracker getPerformanceTracker() {
return performanceTracker;
}
}
/**
* StatsTracker is used to track request level aggregated statistics for a shard. This includes:
* a. currentBytes - Bytes of data that is inflight/processing for a shard.
* b. totalBytes - Total bytes that are processed/completed successfully for a shard.
* c. requestCount - Total number of requests that are processed/completed successfully for a shard.
*/
public static class StatsTracker {
private final AtomicLong currentBytes = new AtomicLong();
private final AtomicLong totalBytes = new AtomicLong();
private final AtomicLong requestCount = new AtomicLong();
public long getCurrentBytes() {
return currentBytes.get();
}
public long incrementCurrentBytes(long bytes) {
return currentBytes.addAndGet(bytes);
}
public long getTotalBytes() {
return totalBytes.get();
}
public long incrementTotalBytes(long bytes) {
return totalBytes.addAndGet(bytes);
}
public long getRequestCount() {
return requestCount.get();
}
public long incrementRequestCount() {
return requestCount.incrementAndGet();
}
}
/**
* RejectionTracker allows tracking the rejection statistics per shard. This includes:
* a. totalRejections - Total number of requests that were rejected for a shard.
* b. nodeLimitsBreachedRejections - Total number of requests that were rejected due to the node level limits breached
* i.e. when a request for a shard could not be served due to node level limit was already reached.
* c. lastSuccessfulRequestLimitsBreachedRejections - Total number of requests that were rejected due to the
* last successful request limits breached for a shard i.e. complete path failure (black-hole).
* d. throughputDegradationLimitsBreachedRejections - Total number of requests that were rejected due to the
* throughput degradation in the request path for a shard i.e. partial failure.
*/
public static class RejectionTracker {
private final AtomicLong totalRejections = new AtomicLong();
private final AtomicLong nodeLimitsBreachedRejections = new AtomicLong();
private final AtomicLong lastSuccessfulRequestLimitsBreachedRejections = new AtomicLong();
private final AtomicLong throughputDegradationLimitsBreachedRejections = new AtomicLong();
public long getTotalRejections() {
return totalRejections.get();
}
public long incrementTotalRejections() {
return totalRejections.incrementAndGet();
}
public long getNodeLimitsBreachedRejections() {
return nodeLimitsBreachedRejections.get();
}
public long incrementNodeLimitsBreachedRejections() {
return nodeLimitsBreachedRejections.incrementAndGet();
}
public long getLastSuccessfulRequestLimitsBreachedRejections() {
return lastSuccessfulRequestLimitsBreachedRejections.get();
}
public long incrementLastSuccessfulRequestLimitsBreachedRejections() {
return lastSuccessfulRequestLimitsBreachedRejections.incrementAndGet();
}
public long getThroughputDegradationLimitsBreachedRejections() {
return throughputDegradationLimitsBreachedRejections.get();
}
public long incrementThroughputDegradationLimitsBreachedRejections() {
return throughputDegradationLimitsBreachedRejections.incrementAndGet();
}
}
/**
* Performance Tracker is used to track the request performance statistics for every operation, per shard. This includes:
* a. latencyInMillis - Total indexing time take by requests that were processed successfully for a shard.
* b. lastSuccessfulRequestTimestamp - Timestamp of last successful request for a shard.
* c. TotalOutstandingRequests - Total requests outstanding for a shard at any given point.
* d. ThroughputMovingAverage - Total moving average throughput value for last N requests.
* e. ThroughputMovingQueue - Queue that holds the last N requests throughput such that there exists a sliding window
* which keeps moving everytime a new request comes. At any given point it tracks last N requests only.
* EWMA cannot be used here as it evaluate the historical average, while here it just needs the average of last N requests.
*/
public static class PerformanceTracker {
private final AtomicLong latencyInMillis = new AtomicLong();
private volatile long lastSuccessfulRequestTimestamp = 0;
private final AtomicLong totalOutstandingRequests = 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
*/
private final AtomicLong throughputMovingAverage = new AtomicLong();
private final ConcurrentLinkedQueue<Double> throughputMovingQueue = new ConcurrentLinkedQueue<>();
public long getLatencyInMillis() {
return latencyInMillis.get();
}
public long addLatencyInMillis(long latency) {
return latencyInMillis.addAndGet(latency);
}
public long getLastSuccessfulRequestTimestamp() {
return lastSuccessfulRequestTimestamp;
}
public void updateLastSuccessfulRequestTimestamp(long timeStamp) {
lastSuccessfulRequestTimestamp = timeStamp;
}
public long getTotalOutstandingRequests() {
return totalOutstandingRequests.get();
}
public long incrementTotalOutstandingRequests() {
return totalOutstandingRequests.incrementAndGet();
}
public void resetTotalOutstandingRequests() {
totalOutstandingRequests.set(0L);
}
public long getThroughputMovingAverage() {
return throughputMovingAverage.get();
}
public long updateThroughputMovingAverage(long newAvg) {
return throughputMovingAverage.getAndSet(newAvg);
}
public boolean addNewThroughout(Double newThroughput) {
return throughputMovingQueue.offer(newThroughput);
}
public Double getFirstThroughput() {
return throughputMovingQueue.poll();
}
public long getThroughputMovingQueueSize() {
return throughputMovingQueue.size();
}
}
/**
* Common operation tracker is used to track values applicable across the operations for a specific shard role. This includes:
* a. currentCombinedCoordinatingAndPrimaryBytes - Bytes of data that is inflight/processing for a shard
* when primary is local to coordinator node. Hence common accounting for coordinator and primary operation.
* b. totalCombinedCoordinatingAndPrimaryBytes - Total bytes that are processed/completed successfully for a shard
* when primary is local to coordinator node. Hence common accounting for coordinator and primary operation.
*/
public static class CommonOperationTracker {
private final AtomicLong currentCombinedCoordinatingAndPrimaryBytes = new AtomicLong();
private final AtomicLong totalCombinedCoordinatingAndPrimaryBytes = new AtomicLong();
public long getCurrentCombinedCoordinatingAndPrimaryBytes() {
return currentCombinedCoordinatingAndPrimaryBytes.get();
}
public long incrementCurrentCombinedCoordinatingAndPrimaryBytes(long bytes) {
return currentCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes);
}
public long getTotalCombinedCoordinatingAndPrimaryBytes() {
return totalCombinedCoordinatingAndPrimaryBytes.get();
}
public long incrementTotalCombinedCoordinatingAndPrimaryBytes(long bytes) {
return totalCombinedCoordinatingAndPrimaryBytes.addAndGet(bytes);
}
}
}

View File

@ -38,7 +38,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.store.AlreadyClosedException;
import org.opensearch.ExceptionsHelper;
import org.opensearch.action.ActionListener;
import org.opensearch.index.IndexingPressure;
import org.opensearch.action.support.ActionFilters;
import org.opensearch.action.support.ActiveShardCount;
import org.opensearch.action.support.WriteResponse;
@ -55,6 +54,7 @@ import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.concurrent.ThreadContext;
import org.opensearch.index.IndexNotFoundException;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.shard.IndexShardClosedException;
import org.opensearch.index.shard.ShardId;
@ -94,7 +94,7 @@ public class RetentionLeaseSyncAction extends
final ThreadPool threadPool,
final ShardStateAction shardStateAction,
final ActionFilters actionFilters,
final IndexingPressure indexingPressure,
final IndexingPressureService indexingPressureService,
final SystemIndices systemIndices) {
super(
settings,
@ -107,7 +107,7 @@ public class RetentionLeaseSyncAction extends
actionFilters,
RetentionLeaseSyncAction.Request::new,
RetentionLeaseSyncAction.Request::new,
ignore -> ThreadPool.Names.MANAGEMENT, false, indexingPressure, systemIndices);
ignore -> ThreadPool.Names.MANAGEMENT, false, indexingPressureService, systemIndices);
}
@Override

View File

@ -0,0 +1,423 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index.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.getShardId().toString();
this.shardIndexingPressureEnforced = shardIndexingPressureEnforced;
totalCombinedCoordinatingAndPrimaryBytes =
shardIndexingPressureTracker.getCommonOperationTracker().getTotalCombinedCoordinatingAndPrimaryBytes();
totalCoordinatingBytes = shardIndexingPressureTracker.getCoordinatingOperationTracker().getStatsTracker().getTotalBytes();
totalPrimaryBytes = shardIndexingPressureTracker.getPrimaryOperationTracker().getStatsTracker().getTotalBytes();
totalReplicaBytes = shardIndexingPressureTracker.getReplicaOperationTracker().getStatsTracker().getTotalBytes();
currentCombinedCoordinatingAndPrimaryBytes =
shardIndexingPressureTracker.getCommonOperationTracker().getCurrentCombinedCoordinatingAndPrimaryBytes();
currentCoordinatingBytes = shardIndexingPressureTracker.getCoordinatingOperationTracker().getStatsTracker().getCurrentBytes();
currentPrimaryBytes = shardIndexingPressureTracker.getPrimaryOperationTracker().getStatsTracker().getCurrentBytes();
currentReplicaBytes = shardIndexingPressureTracker.getReplicaOperationTracker().getStatsTracker().getCurrentBytes();
totalCoordinatingCount = shardIndexingPressureTracker.getCoordinatingOperationTracker().getStatsTracker().getRequestCount();
totalPrimaryCount = shardIndexingPressureTracker.getPrimaryOperationTracker().getStatsTracker().getRequestCount();
totalReplicaCount = shardIndexingPressureTracker.getReplicaOperationTracker().getStatsTracker().getRequestCount();
coordinatingRejections = shardIndexingPressureTracker.getCoordinatingOperationTracker().getRejectionTracker().getTotalRejections();
coordinatingNodeLimitsBreachedRejections = shardIndexingPressureTracker.getCoordinatingOperationTracker()
.getRejectionTracker().getNodeLimitsBreachedRejections();
coordinatingLastSuccessfulRequestLimitsBreachedRejections = shardIndexingPressureTracker.getCoordinatingOperationTracker()
.getRejectionTracker().getLastSuccessfulRequestLimitsBreachedRejections();
coordinatingThroughputDegradationLimitsBreachedRejections = shardIndexingPressureTracker.getCoordinatingOperationTracker()
.getRejectionTracker().getThroughputDegradationLimitsBreachedRejections();
primaryRejections = shardIndexingPressureTracker.getPrimaryOperationTracker().getRejectionTracker().getTotalRejections();
primaryNodeLimitsBreachedRejections = shardIndexingPressureTracker.getPrimaryOperationTracker()
.getRejectionTracker().getNodeLimitsBreachedRejections();
primaryLastSuccessfulRequestLimitsBreachedRejections = shardIndexingPressureTracker.getPrimaryOperationTracker()
.getRejectionTracker().getLastSuccessfulRequestLimitsBreachedRejections();
primaryThroughputDegradationLimitsBreachedRejections = shardIndexingPressureTracker.getPrimaryOperationTracker()
.getRejectionTracker().getThroughputDegradationLimitsBreachedRejections();
replicaRejections = shardIndexingPressureTracker.getReplicaOperationTracker().getRejectionTracker().getTotalRejections();
replicaNodeLimitsBreachedRejections = shardIndexingPressureTracker.getReplicaOperationTracker()
.getRejectionTracker().getNodeLimitsBreachedRejections();
replicaLastSuccessfulRequestLimitsBreachedRejections = shardIndexingPressureTracker.getReplicaOperationTracker()
.getRejectionTracker().getLastSuccessfulRequestLimitsBreachedRejections();
replicaThroughputDegradationLimitsBreachedRejections = shardIndexingPressureTracker.getReplicaOperationTracker()
.getRejectionTracker().getThroughputDegradationLimitsBreachedRejections();
coordinatingTimeInMillis = shardIndexingPressureTracker.getCoordinatingOperationTracker().getPerformanceTracker()
.getLatencyInMillis();
primaryTimeInMillis = shardIndexingPressureTracker.getPrimaryOperationTracker().getPerformanceTracker()
.getLatencyInMillis();
replicaTimeInMillis = shardIndexingPressureTracker.getReplicaOperationTracker().getPerformanceTracker()
.getLatencyInMillis();
coordinatingLastSuccessfulRequestTimestampInMillis = shardIndexingPressureTracker.getCoordinatingOperationTracker()
.getPerformanceTracker().getLastSuccessfulRequestTimestamp();
primaryLastSuccessfulRequestTimestampInMillis = shardIndexingPressureTracker.getPrimaryOperationTracker()
.getPerformanceTracker().getLastSuccessfulRequestTimestamp();
replicaLastSuccessfulRequestTimestampInMillis = shardIndexingPressureTracker.getReplicaOperationTracker()
.getPerformanceTracker().getLastSuccessfulRequestTimestamp();
currentPrimaryAndCoordinatingLimits = shardIndexingPressureTracker.getPrimaryAndCoordinatingLimits();
currentReplicaLimits = shardIndexingPressureTracker.getReplicaLimits();
}
@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();
}
}

View File

@ -0,0 +1,106 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index.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<ShardId, IndexingPressurePerShardStats> 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++) {
ShardId shardId = new ShardId(in);
IndexingPressurePerShardStats shardStats = new IndexingPressurePerShardStats(in);
shardIndexingPressureStore.put(shardId, shardStats);
}
totalNodeLimitsBreachedRejections = in.readVLong();
totalLastSuccessfulRequestLimitsBreachedRejections = in.readVLong();
totalThroughputDegradationLimitsBreachedRejections = in.readVLong();
shardIndexingPressureEnabled = in.readBoolean();
shardIndexingPressureEnforced = in.readBoolean();
}
public ShardIndexingPressureStats(Map<ShardId, IndexingPressurePerShardStats> 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<ShardId, IndexingPressurePerShardStats> entry : shardIndexingPressureStore.entrySet()) {
entry.getKey().writeTo(out);
entry.getValue().writeTo(out);
}
out.writeVLong(totalNodeLimitsBreachedRejections);
out.writeVLong(totalLastSuccessfulRequestLimitsBreachedRejections);
out.writeVLong(totalThroughputDegradationLimitsBreachedRejections);
out.writeBoolean(shardIndexingPressureEnabled);
out.writeBoolean(shardIndexingPressureEnforced);
}
public IndexingPressurePerShardStats getIndexingPressureShardStats(ShardId shardId) {
return shardIndexingPressureStore.get(shardId);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException {
builder.startObject("shard_indexing_pressure");
builder.startObject("stats");
for (Map.Entry<ShardId, IndexingPressurePerShardStats> 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);
}
}
}

View File

@ -36,6 +36,7 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.SetOnce;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.watcher.ResourceWatcherService;
import org.opensearch.Assertions;
import org.opensearch.Build;
@ -120,7 +121,6 @@ import org.opensearch.gateway.MetaStateService;
import org.opensearch.gateway.PersistedClusterStateService;
import org.opensearch.http.HttpServerTransport;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.IndexingPressure;
import org.opensearch.index.analysis.AnalysisRegistry;
import org.opensearch.index.engine.EngineFactory;
import org.opensearch.indices.IndicesModule;
@ -218,6 +218,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import static java.util.stream.Collectors.toList;
import static org.opensearch.index.ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY;
/**
* A node represent a node within a cluster ({@code cluster.name}). The {@link #client()} can be used
@ -317,6 +318,11 @@ public class Node implements Closeable {
Settings tmpSettings = Settings.builder().put(initialEnvironment.settings())
.put(Client.CLIENT_TYPE_SETTING_S.getKey(), CLIENT_TYPE).build();
// Enabling shard indexing backpressure node-attribute
tmpSettings = Settings.builder().put(tmpSettings)
.put(NODE_ATTRIBUTES.getKey() + SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY, "true")
.build();
final JvmInfo jvmInfo = JvmInfo.jvmInfo();
logger.info(
"version[{}], pid[{}], build[{}/{}/{}], OS[{}/{}/{}], JVM[{}/{}/{}/{}]",
@ -599,7 +605,10 @@ public class Node implements Closeable {
final SearchTransportService searchTransportService = new SearchTransportService(transportService,
SearchExecutionStatsCollector.makeWrapper(responseCollectorService));
final HttpServerTransport httpServerTransport = newHttpTransport(networkModule);
final IndexingPressure indexingLimits = new IndexingPressure(settings);
final IndexingPressureService indexingPressureService = new IndexingPressureService(settings, clusterService);
// Going forward, IndexingPressureService will have required constructs for exposing listeners/interfaces for plugin
// development. Then we can deprecate Getter and Setter for IndexingPressureService in ClusterService (#478).
clusterService.setIndexingPressureService(indexingPressureService);
final RecoverySettings recoverySettings = new RecoverySettings(settings, settingsModule.getClusterSettings());
RepositoriesModule repositoriesModule = new RepositoriesModule(this.environment,
@ -628,7 +637,7 @@ public class Node implements Closeable {
this.nodeService = new NodeService(settings, threadPool, monitorService, discoveryModule.getDiscovery(),
transportService, indicesService, pluginsService, circuitBreakerService, scriptService,
httpServerTransport, ingestService, clusterService, settingsModule.getSettingsFilter(), responseCollectorService,
searchTransportService, indexingLimits, searchModule.getValuesSourceRegistry().getUsageService());
searchTransportService, indexingPressureService, searchModule.getValuesSourceRegistry().getUsageService());
final SearchService searchService = newSearchService(clusterService, indicesService,
threadPool, scriptService, bigArrays, searchModule.getFetchPhase(),
@ -664,7 +673,7 @@ public class Node implements Closeable {
b.bind(ScriptService.class).toInstance(scriptService);
b.bind(AnalysisRegistry.class).toInstance(analysisModule.getAnalysisRegistry());
b.bind(IngestService.class).toInstance(ingestService);
b.bind(IndexingPressure.class).toInstance(indexingLimits);
b.bind(IndexingPressureService.class).toInstance(indexingPressureService);
b.bind(UsageService.class).toInstance(usageService);
b.bind(AggregationUsageService.class).toInstance(searchModule.getValuesSourceRegistry().getUsageService());
b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry);

View File

@ -32,7 +32,6 @@
package org.opensearch.node;
import org.opensearch.index.IndexingPressure;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.Build;
import org.opensearch.Version;
@ -46,6 +45,7 @@ import org.opensearch.common.settings.Settings;
import org.opensearch.common.settings.SettingsFilter;
import org.opensearch.discovery.Discovery;
import org.opensearch.http.HttpServerTransport;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.indices.IndicesService;
import org.opensearch.indices.breaker.CircuitBreakerService;
import org.opensearch.ingest.IngestService;
@ -74,7 +74,7 @@ public class NodeService implements Closeable {
private final HttpServerTransport httpServerTransport;
private final ResponseCollectorService responseCollectorService;
private final SearchTransportService searchTransportService;
private final IndexingPressure indexingPressure;
private final IndexingPressureService indexingPressureService;
private final AggregationUsageService aggregationUsageService;
private final Discovery discovery;
@ -84,7 +84,7 @@ public class NodeService implements Closeable {
CircuitBreakerService circuitBreakerService, ScriptService scriptService,
@Nullable HttpServerTransport httpServerTransport, IngestService ingestService, ClusterService clusterService,
SettingsFilter settingsFilter, ResponseCollectorService responseCollectorService,
SearchTransportService searchTransportService, IndexingPressure indexingPressure,
SearchTransportService searchTransportService, IndexingPressureService indexingPressureService,
AggregationUsageService aggregationUsageService) {
this.settings = settings;
this.threadPool = threadPool;
@ -100,7 +100,7 @@ public class NodeService implements Closeable {
this.scriptService = scriptService;
this.responseCollectorService = responseCollectorService;
this.searchTransportService = searchTransportService;
this.indexingPressure = indexingPressure;
this.indexingPressureService = indexingPressureService;
this.aggregationUsageService = aggregationUsageService;
clusterService.addStateApplier(ingestService);
}
@ -125,7 +125,7 @@ public class NodeService implements Closeable {
public NodeStats stats(CommonStatsFlags indices, boolean os, boolean process, boolean jvm, boolean threadPool,
boolean fs, boolean transport, boolean http, boolean circuitBreaker,
boolean script, boolean discoveryStats, boolean ingest, boolean adaptiveSelection, boolean scriptCache,
boolean indexingPressure) {
boolean indexingPressure, boolean shardIndexingPressure) {
// for indices stats we want to include previous allocated shards stats as well (it will
// only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats)
return new NodeStats(transportService.getLocalNode(), System.currentTimeMillis(),
@ -143,7 +143,8 @@ public class NodeService implements Closeable {
ingest ? ingestService.stats() : null,
adaptiveSelection ? responseCollectorService.getAdaptiveStats(searchTransportService.getPendingSearchRequests()) : null,
scriptCache ? scriptService.cacheStats() : null,
indexingPressure ? this.indexingPressure.stats() : null
indexingPressure ? this.indexingPressureService.nodeStats() : null,
shardIndexingPressure ? this.indexingPressureService.shardStats(indices) : null
);
}

View File

@ -189,6 +189,13 @@ public class RestNodesStatsAction extends BaseRestHandler {
if (nodesStatsRequest.indices().isSet(Flag.Segments)) {
nodesStatsRequest.indices().includeSegmentFileSizes(request.paramAsBoolean("include_segment_file_sizes", false));
}
if (request.hasParam("include_all")) {
nodesStatsRequest.indices().includeAllShardIndexingPressureTrackers(request.paramAsBoolean("include_all", false));
}
if (request.hasParam("top")) {
nodesStatsRequest.indices().includeOnlyTopIndexingPressureMetrics(request.paramAsBoolean("top", false));
}
return channel -> client.admin().cluster().nodesStats(nodesStatsRequest, new NodesResponseRestListener<>(channel));
}

View File

@ -566,7 +566,7 @@ public class NodeStatsTests extends OpenSearchTestCase {
//TODO NodeIndicesStats are not tested here, way too complicated to create, also they need to be migrated to Writeable yet
return new NodeStats(node, randomNonNegativeLong(), null, osStats, processStats, jvmStats, threadPoolStats,
fsInfo, transportStats, httpStats, allCircuitBreakerStats, scriptStats, discoveryStats,
ingestStats, adaptiveSelectionStats, scriptCacheStats, null);
ingestStats, adaptiveSelectionStats, scriptCacheStats, null, null);
}
private IngestStats.Stats getPipelineStats(List<IngestStats.PipelineStat> pipelineStats, String id) {

View File

@ -44,13 +44,14 @@ import org.opensearch.cluster.metadata.Metadata;
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;
import org.opensearch.common.util.concurrent.OpenSearchExecutors;
import org.opensearch.index.IndexNotFoundException;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.VersionType;
import org.opensearch.index.IndexingPressure;
import org.opensearch.indices.SystemIndices;
import org.opensearch.tasks.Task;
import org.opensearch.test.OpenSearchTestCase;
@ -138,7 +139,9 @@ public class TransportBulkActionIndicesThatCannotBeCreatedTests extends OpenSear
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 IndexingPressureService(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<BulkResponse> listener,
AtomicArray<BulkItemResponse> responses, Map<String, IndexNotFoundException> indicesThatCannotBeCreated) {

View File

@ -67,7 +67,7 @@ import org.opensearch.common.util.concurrent.OpenSearchExecutors;
import org.opensearch.common.util.concurrent.ThreadContext;
import org.opensearch.index.IndexNotFoundException;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.IndexingPressure;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.indices.SystemIndices;
import org.opensearch.ingest.IngestService;
import org.opensearch.tasks.Task;
@ -163,7 +163,8 @@ public class TransportBulkActionIngestTests extends OpenSearchTestCase {
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 IndexingPressureService(SETTINGS, new ClusterService(SETTINGS, new ClusterSettings(SETTINGS,
ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null)), new SystemIndices(emptyMap())
);
}
@Override

View File

@ -58,7 +58,7 @@ import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException;
import org.opensearch.index.IndexNotFoundException;
import org.opensearch.index.IndexingPressure;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.VersionType;
import org.opensearch.indices.SystemIndexDescriptor;
import org.opensearch.indices.SystemIndices;
@ -105,7 +105,7 @@ public class TransportBulkActionTests extends OpenSearchTestCase {
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 IndexingPressureService(Settings.EMPTY, clusterService), new SystemIndices(emptyMap()));
}
@Override

View File

@ -55,8 +55,8 @@ import org.opensearch.common.util.concurrent.AtomicArray;
import org.opensearch.common.util.concurrent.ThreadContext;
import org.opensearch.common.xcontent.XContentType;
import org.opensearch.index.IndexNotFoundException;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.rest.action.document.RestBulkAction;
import org.opensearch.index.IndexingPressure;
import org.opensearch.indices.SystemIndices;
import org.opensearch.tasks.Task;
import org.opensearch.test.OpenSearchTestCase;
@ -266,7 +266,7 @@ public class TransportBulkActionTookTests extends OpenSearchTestCase {
actionFilters,
indexNameExpressionResolver,
autoCreateIndex,
new IndexingPressure(Settings.EMPTY),
new IndexingPressureService(Settings.EMPTY, clusterService),
new SystemIndices(emptyMap()),
relativeTimeProvider);
}

View File

@ -33,8 +33,6 @@ package org.opensearch.action.resync;
import org.opensearch.Version;
import org.opensearch.action.ActionListener;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.IndexingPressure;
import org.opensearch.action.support.ActionFilters;
import org.opensearch.action.support.PlainActionFuture;
import org.opensearch.cluster.ClusterState;
@ -53,6 +51,8 @@ import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.PageCacheRecycler;
import org.opensearch.index.Index;
import org.opensearch.index.IndexService;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.shard.ReplicationGroup;
import org.opensearch.index.shard.ShardId;
@ -162,7 +162,7 @@ public class TransportResyncReplicationActionTests extends OpenSearchTestCase {
final TransportResyncReplicationAction action = new TransportResyncReplicationAction(Settings.EMPTY, transportService,
clusterService, indexServices, threadPool, shardStateAction, new ActionFilters(new HashSet<>()),
new IndexingPressure(Settings.EMPTY), new SystemIndices(emptyMap()));
new IndexingPressureService(Settings.EMPTY, clusterService), new SystemIndices(emptyMap()));
assertThat(action.globalBlockLevel(), nullValue());
assertThat(action.indexBlockLevel(), nullValue());

View File

@ -0,0 +1,487 @@
/*
* Copyright OpenSearch Contributors.
* SPDX-License-Identifier: Apache-2.0
*/
package org.opensearch.action.support.replication;
import org.opensearch.action.ActionListener;
import org.opensearch.action.admin.indices.stats.CommonStatsFlags;
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.IndexingPressureService;
import org.opensearch.index.ShardIndexingPressureSettings;
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.stats.IndexingPressurePerShardStats;
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.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.Objects;
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.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.when;
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 Releasable releasable;
private IndexingPressureService indexingPressureService;
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);
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();
final Settings settings = Settings.builder().put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false)
.build();
this.indexingPressureService = new IndexingPressureService(settings, clusterService);
TestAction action = new TestAction(settings, "internal:testAction", transportService, clusterService,
shardStateAction, threadPool);
action.handleReplicaRequest(
new TransportReplicationAction.ConcreteReplicaRequest<>(
new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(),
randomNonNegativeLong(), randomNonNegativeLong()),
createTransportChannel(new PlainActionFuture<>()), task);
IndexingPressurePerShardStats shardStats =
this.indexingPressureService.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertPhase(task, "finished");
assertTrue(Objects.isNull(shardStats));
}
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();
final Settings settings = Settings.builder().put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.build();
this.indexingPressureService = new IndexingPressureService(settings, clusterService);
TestAction action = new TestAction(settings, "internal:testAction", transportService, clusterService,
shardStateAction, threadPool);
action.handleReplicaRequest(
new TransportReplicationAction.ConcreteReplicaRequest<>(
new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(),
randomNonNegativeLong(), randomNonNegativeLong()),
createTransportChannel(new PlainActionFuture<>()), task);
CommonStatsFlags statsFlag = new CommonStatsFlags();
statsFlag.includeAllShardIndexingPressureTrackers(true);
IndexingPressurePerShardStats shardStats =
this.indexingPressureService.shardStats(statsFlag).getIndexingPressureShardStats(shardId);
assertPhase(task, "finished");
assertTrue(!Objects.isNull(shardStats));
assertEquals(100, shardStats.getTotalReplicaBytes());
}
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();
final Settings settings =
Settings.builder().put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false).build();
this.indexingPressureService = new IndexingPressureService(settings, clusterService);
TestAction action = new TestAction(settings, "internal:testActionWithExceptions", transportService, clusterService,
shardStateAction, threadPool);
action.handlePrimaryRequest(
new TransportReplicationAction.ConcreteReplicaRequest<>(
new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(),
randomNonNegativeLong(), randomNonNegativeLong()),
createTransportChannel(new PlainActionFuture<>()), task);
IndexingPressurePerShardStats shardStats =
this.indexingPressureService.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertPhase(task, "finished");
assertTrue(Objects.isNull(shardStats));
}
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();
final Settings settings =
Settings.builder().put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true).build();
this.indexingPressureService = new IndexingPressureService(settings, clusterService);
TestAction action = new TestAction(settings, "internal:testActionWithExceptions", transportService, clusterService,
shardStateAction, threadPool);
action.handlePrimaryRequest(
new TransportReplicationAction.ConcreteReplicaRequest<>(
new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(),
randomNonNegativeLong(), randomNonNegativeLong()),
createTransportChannel(new PlainActionFuture<>()), task);
CommonStatsFlags statsFlag = new CommonStatsFlags();
statsFlag.includeAllShardIndexingPressureTrackers(true);
IndexingPressurePerShardStats shardStats =
this.indexingPressureService.shardStats(statsFlag).getIndexingPressureShardStats(shardId);
assertPhase(task, "finished");
assertTrue(!Objects.isNull(shardStats));
assertEquals(100, shardStats.getTotalPrimaryBytes());
}
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();
final Settings settings = Settings.builder().put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false)
.build();
this.indexingPressureService = new IndexingPressureService(settings, clusterService);
TestAction action = new TestAction(settings, "internal:testAction", transportService, clusterService,
shardStateAction, threadPool);
action.handlePrimaryRequest(
new TransportReplicationAction.ConcreteShardRequest<>(
new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(),
true, true),
createTransportChannel(new PlainActionFuture<>()), task);
IndexingPressurePerShardStats shardStats =
this.indexingPressureService.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertPhase(task, "finished");
assertTrue(Objects.isNull(shardStats));
}
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();
final Settings settings = Settings.builder().put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.build();
this.indexingPressureService = new IndexingPressureService(settings, clusterService);
TestAction action = new TestAction(settings, "internal:testAction", transportService, clusterService,
shardStateAction, threadPool);
action.handlePrimaryRequest(
new TransportReplicationAction.ConcreteShardRequest<>(
new TestRequest(), replicaRouting.allocationId().getId(), randomNonNegativeLong(),
true, true),
createTransportChannel(new PlainActionFuture<>()), task);
CommonStatsFlags statsFlag = new CommonStatsFlags();
statsFlag.includeAllShardIndexingPressureTrackers(true);
IndexingPressurePerShardStats shardStats =
this.indexingPressureService.shardStats(statsFlag).getIndexingPressureShardStats(shardId);
assertPhase(task, "finished");
assertTrue(!Objects.isNull(shardStats));
}
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<String> phaseMatcher) {
if (task != null) {
assertThat(task.getPhase(), phaseMatcher);
}
}
private class TestAction extends TransportWriteAction<TestRequest, TestRequest, TestResponse> {
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,
TransportWriteActionForIndexingPressureTests.this.indexingPressureService, new SystemIndices(emptyMap()));
}
@Override
protected TestResponse newResponseInstance(StreamInput in) throws IOException {
return new TestResponse();
}
@Override
protected long primaryOperationSize(TestRequest request) {
return 100;
}
@Override
protected long replicaOperationSize(TestRequest request) {
return 100;
}
@Override
protected void dispatchedShardOperationOnPrimary(
TestRequest request, IndexShard primary, ActionListener<PrimaryResult<TestRequest, TestResponse>> listener) {
ActionListener.completeWith(listener, () -> new WritePrimaryResult<>(request, new TestResponse(), location, null, primary,
logger));
}
@Override
protected void dispatchedShardOperationOnReplica(TestRequest request, IndexShard replica, ActionListener<ReplicaResult> listener) {
ActionListener.completeWith(listener, () -> new WriteReplicaResult<>(request, location, null, replica, logger));
}
}
private static class TestRequest extends ReplicatedWriteRequest<TestRequest> {
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<Releasable> callback = (ActionListener<Releasable>) 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<Releasable> callback = (ActionListener<Releasable>) 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<TestResponse> 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";
}
};
}
}

View File

@ -34,7 +34,6 @@ package org.opensearch.action.support.replication;
import org.opensearch.OpenSearchException;
import org.opensearch.action.ActionListener;
import org.opensearch.index.IndexingPressure;
import org.opensearch.action.support.ActionFilters;
import org.opensearch.action.support.ActionTestUtils;
import org.opensearch.action.support.PlainActionFuture;
@ -56,6 +55,7 @@ import org.opensearch.common.lease.Releasable;
import org.opensearch.common.settings.Settings;
import org.opensearch.index.Index;
import org.opensearch.index.IndexService;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.shard.ShardId;
import org.opensearch.index.shard.ShardNotFoundException;
@ -382,7 +382,7 @@ public class TransportWriteActionTests extends OpenSearchTestCase {
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 IndexingPressureService(Settings.EMPTY, TransportWriteActionTests.this.clusterService), new SystemIndices(emptyMap()));
this.withDocumentFailureOnPrimary = withDocumentFailureOnPrimary;
this.withDocumentFailureOnReplica = withDocumentFailureOnReplica;
}
@ -392,7 +392,7 @@ public class TransportWriteActionTests extends OpenSearchTestCase {
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 IndexingPressureService(settings, clusterService), new SystemIndices(emptyMap()));
this.withDocumentFailureOnPrimary = false;
this.withDocumentFailureOnReplica = false;
}

View File

@ -166,13 +166,13 @@ public class DiskUsageTests extends OpenSearchTestCase {
List<NodeStats> nodeStats = Arrays.asList(
new NodeStats(new DiscoveryNode("node_1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
null,null,null,null,null,new FsInfo(0, null, node1FSInfo), null,null,null,null,null, null, null,
null, null),
null, null, null),
new NodeStats(new DiscoveryNode("node_2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, null, node2FSInfo), null,null,null,null,null, null, null,
null, null),
null, null, null),
new NodeStats(new DiscoveryNode("node_3", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, null, node3FSInfo), null,null,null,null,null, null, null,
null, null)
null, null, null)
);
InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvaiableUsages, newMostAvaiableUsages);
DiskUsage leastNode_1 = newLeastAvaiableUsages.get("node_1");
@ -210,13 +210,13 @@ public class DiskUsageTests extends OpenSearchTestCase {
List<NodeStats> nodeStats = Arrays.asList(
new NodeStats(new DiscoveryNode("node_1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
null,null,null,null,null,new FsInfo(0, null, node1FSInfo), null,null,null,null,null, null, null,
null, null),
null, null, null),
new NodeStats(new DiscoveryNode("node_2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, null, node2FSInfo), null,null,null,null,null, null, null,
null, null),
null, null, null),
new NodeStats(new DiscoveryNode("node_3", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, null, node3FSInfo), null,null,null,null,null, null, null,
null, null)
null, null, null)
);
InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvailableUsages, newMostAvailableUsages);
DiskUsage leastNode_1 = newLeastAvailableUsages.get("node_1");

View File

@ -0,0 +1,163 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index;
import org.junit.Before;
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.index.shard.ShardId;
import org.opensearch.index.stats.IndexingPressurePerShardStats;
import org.opensearch.index.stats.IndexingPressureStats;
import org.opensearch.test.OpenSearchTestCase;
public class IndexingPressureServiceTests 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(), "20ms")
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true)
.put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 100)
.build();
private ClusterSettings clusterSettings;
private ClusterService clusterService;
@Before
public void beforeTest() {
clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
clusterService = new ClusterService(settings, clusterSettings, null);
}
public void testCoordinatingOperationForShardIndexingPressure() {
IndexingPressureService service = new IndexingPressureService(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
Releasable releasable = service.markCoordinatingOperationStarted(shardId, 1024, false);
IndexingPressurePerShardStats shardStats = service.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertEquals(1024, shardStats.getCurrentCoordinatingBytes());
releasable.close();
}
public void testCoordinatingOperationForIndexingPressure() {
IndexingPressureService service = new IndexingPressureService(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
Settings.Builder updated = Settings.builder();
clusterSettings.updateDynamicSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false).build(),
Settings.builder().put(settings), updated, getTestClass().getName());
clusterSettings.applySettings(updated.build());
Releasable releasable = service.markCoordinatingOperationStarted(1024, false);
IndexingPressurePerShardStats shardStats = service.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertNull(shardStats);
IndexingPressureStats nodeStats = service.nodeStats();
assertEquals(1024, nodeStats.getCurrentCoordinatingBytes());
releasable.close();
}
public void testPrimaryOperationForShardIndexingPressure() {
IndexingPressureService service = new IndexingPressureService(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
Releasable releasable = service.markPrimaryOperationStarted(shardId, 1024, false);
IndexingPressurePerShardStats shardStats = service.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertEquals(1024, shardStats.getCurrentPrimaryBytes());
releasable.close();
}
public void testPrimaryOperationForIndexingPressure() {
IndexingPressureService service = new IndexingPressureService(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
Settings.Builder updated = Settings.builder();
clusterSettings.updateDynamicSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false).build(),
Settings.builder().put(settings), updated, getTestClass().getName());
clusterSettings.applySettings(updated.build());
Releasable releasable = service.markPrimaryOperationStarted(shardId, 1024, false);
IndexingPressurePerShardStats shardStats = service.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertNull(shardStats);
IndexingPressureStats nodeStats = service.nodeStats();
assertEquals(1024, nodeStats.getCurrentPrimaryBytes());
releasable.close();
}
public void testLocalPrimaryOperationForShardIndexingPressure() {
IndexingPressureService service = new IndexingPressureService(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
Releasable releasable = service.markPrimaryOperationLocalToCoordinatingNodeStarted(shardId, 1024);
IndexingPressurePerShardStats shardStats = service.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertEquals(1024, shardStats.getCurrentPrimaryBytes());
releasable.close();
}
public void testLocalPrimaryOperationForIndexingPressure() {
IndexingPressureService service = new IndexingPressureService(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
Settings.Builder updated = Settings.builder();
clusterSettings.updateDynamicSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false).build(),
Settings.builder().put(settings), updated, getTestClass().getName());
clusterSettings.applySettings(updated.build());
Releasable releasable = service.markPrimaryOperationLocalToCoordinatingNodeStarted(shardId, 1024);
IndexingPressurePerShardStats shardStats = service.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertNull(shardStats);
IndexingPressureStats nodeStats = service.nodeStats();
assertEquals(1024, nodeStats.getCurrentPrimaryBytes());
releasable.close();
}
public void testReplicaOperationForShardIndexingPressure() {
IndexingPressureService service = new IndexingPressureService(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
Releasable releasable = service.markReplicaOperationStarted(shardId, 1024, false);
IndexingPressurePerShardStats shardStats = service.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertEquals(1024, shardStats.getCurrentReplicaBytes());
releasable.close();
}
public void testReplicaOperationForIndexingPressure() {
IndexingPressureService service = new IndexingPressureService(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
Settings.Builder updated = Settings.builder();
clusterSettings.updateDynamicSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false).build(),
Settings.builder().put(settings), updated, getTestClass().getName());
clusterSettings.applySettings(updated.build());
Releasable releasable = service.markReplicaOperationStarted(shardId, 1024, false);
IndexingPressurePerShardStats shardStats = service.shardStats(CommonStatsFlags.ALL).getIndexingPressureShardStats(shardId);
assertNull(shardStats);
IndexingPressureStats nodeStats = service.nodeStats();
assertEquals(1024, nodeStats.getCurrentReplicaBytes());
releasable.close();
}
}

View File

@ -40,7 +40,8 @@ import org.opensearch.test.OpenSearchTestCase;
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();
public void testMemoryBytesMarkedAndReleased() {
IndexingPressure indexingPressure = new IndexingPressure(settings);

View File

@ -0,0 +1,851 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index;
import org.hamcrest.Matchers;
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 java.util.concurrent.atomic.AtomicInteger;
public class ShardIndexingPressureConcurrentExecutionTests 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(), "20ms")
.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 enum OperationType { COORDINATING, PRIMARY, REPLICA }
public void testCoordinatingPrimaryThreadedUpdateToShardLimits() throws Exception {
final int NUM_THREADS = scaledRandomIntBetween(100, 500);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId1 = new ShardId(index, 0);
boolean randomBoolean = randomBoolean();
Releasable[] releasable;
if (randomBoolean) {
releasable = fireConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 15, OperationType.COORDINATING);
} else {
releasable = fireConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 15, OperationType.PRIMARY);
}
if(randomBoolean) {
assertEquals(NUM_THREADS * 15, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentCoordinatingBytes());
} else {
assertEquals(NUM_THREADS * 15, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentPrimaryBytes());
}
assertEquals(NUM_THREADS * 15, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertTrue((double) (NUM_THREADS * 15) / shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentPrimaryAndCoordinatingLimits() < 0.95);
assertTrue((double) (NUM_THREADS * 15) / shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentPrimaryAndCoordinatingLimits() > 0.75);
for (int i = 0; i < NUM_THREADS; i++) {
releasable[i].close();
}
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId1 = new ShardId(index, 0);
Releasable[] releasable = fireConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 15, OperationType.REPLICA);
assertEquals(NUM_THREADS * 15, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentReplicaBytes());
assertTrue((double)(NUM_THREADS * 15) / shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentReplicaLimits() < 0.95);
assertTrue((double)(NUM_THREADS * 15) / shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentReplicaLimits() > 0.75);
for (int i = 0; i < NUM_THREADS; i++) {
releasable[i].close();
}
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId1 = new ShardId(index, 0);
boolean randomBoolean = randomBoolean();
if (randomBoolean) {
fireAndCompleteConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 100, OperationType.COORDINATING);
} else {
fireAndCompleteConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 100, OperationType.PRIMARY);
}
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId1 = new ShardId(index, 0);
fireAndCompleteConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 100, OperationType.REPLICA);
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId1 = new ShardId(index, 0);
boolean randomBoolean = randomBoolean();
if (randomBoolean) {
fireAllThenCompleteConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 15, OperationType.COORDINATING);
} else {
fireAllThenCompleteConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 15, OperationType.PRIMARY);
}
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId1 = new ShardId(index, 0);
fireAllThenCompleteConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 15, OperationType.REPLICA);
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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 = shardIndexingPressure.stats();
ShardIndexingPressureStats shardStats = shardIndexingPressure.shardStats();
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 (Releasable releasable : releasables) {
if (releasable != null) {
releasable.close();
}
}
nodeStats = shardIndexingPressure.stats();
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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 = shardIndexingPressure.stats();
assertEquals(rejectionCount.get(), nodeStats.getReplicaRejections());
assertTrue(nodeStats.getCurrentReplicaBytes() < 50 * 300);
ShardIndexingPressureStats shardStats = shardIndexingPressure.shardStats();
assertTrue(shardStats.getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes() < 50 * 300);
for (Releasable releasable : releasables) {
if (releasable != null) {
releasable.close();
}
}
nodeStats = shardIndexingPressure.stats();
assertEquals(rejectionCount.get(), nodeStats.getReplicaRejections());
assertEquals(0, nodeStats.getCurrentReplicaBytes());
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "new_uuid");
ShardId shardId1 = new ShardId(index, 0);
boolean randomBoolean = randomBoolean();
Releasable[] releasables;
if(randomBoolean) {
releasables = fireConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 15, OperationType.COORDINATING);
} else {
releasables = fireConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 15, OperationType.PRIMARY);
}
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats()
.getIndexingPressureShardStats(shardId1);
assertThat(shardStoreStats.getCurrentPrimaryAndCoordinatingLimits(), Matchers.greaterThan(100L));
CommonStatsFlags statsFlag = new CommonStatsFlags();
statsFlag.includeAllShardIndexingPressureTrackers(true);
IndexingPressurePerShardStats shardStoreStats2 = shardIndexingPressure.shardStats(statsFlag)
.getIndexingPressureShardStats(shardId1);;
assertEquals(shardStoreStats.getCurrentPrimaryAndCoordinatingLimits(), shardStoreStats2
.getCurrentPrimaryAndCoordinatingLimits());
statsFlag.includeOnlyTopIndexingPressureMetrics(true);
assertNull(shardIndexingPressure.shardStats(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.shardStats().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.shardStats(statsFlag).getIndexingPressureShardStats(shardId1);
assertEquals(shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1)
.getCurrentPrimaryAndCoordinatingLimits(),
shardStoreStats2.getCurrentPrimaryAndCoordinatingLimits());
statsFlag.includeAllShardIndexingPressureTrackers(false);
assertNull(shardIndexingPressure.shardStats(statsFlag).getIndexingPressureShardStats(shardId1));
}
public void testReplicaConcurrentUpdatesOnShardIndexingPressureTrackerObjects() throws Exception {
final int NUM_THREADS = scaledRandomIntBetween(100, 400);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "new_uuid");
ShardId shardId1 = new ShardId(index, 0);
final Releasable[] releasables = fireConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 20, OperationType.REPLICA);
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats()
.getIndexingPressureShardStats(shardId1);
assertThat(shardStoreStats.getCurrentReplicaLimits(), Matchers.greaterThan(100L));
CommonStatsFlags statsFlag = new CommonStatsFlags();
statsFlag.includeAllShardIndexingPressureTrackers(true);
IndexingPressurePerShardStats shardStoreStats2 = shardIndexingPressure.shardStats(statsFlag)
.getIndexingPressureShardStats(shardId1);;
assertEquals(shardStoreStats.getCurrentReplicaLimits(), shardStoreStats2.getCurrentReplicaLimits());
statsFlag.includeOnlyTopIndexingPressureMetrics(true);
assertNull(shardIndexingPressure.shardStats(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.shardStats().getIndexingPressureShardStats(shardId1);
assertNull(shardStoreStats);
assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaBytes());
assertEquals(15, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits());
shardStoreStats2 = shardIndexingPressure.shardStats(statsFlag).getIndexingPressureShardStats(shardId1);;
assertEquals(shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits(),
shardStoreStats2.getCurrentReplicaLimits());
statsFlag.includeAllShardIndexingPressureTrackers(false);
assertNull(shardIndexingPressure.shardStats(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(), 80)
.build();
final int NUM_THREADS = scaledRandomIntBetween(80, 100);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId1 = new ShardId(index, 0);
boolean randomBoolean = randomBoolean();
//Generating a concurrent + sequential load to have a fair throughput
if (randomBoolean) {
fireConcurrentAndParallelRequestsForUniformThroughPut(NUM_THREADS, shardIndexingPressure, shardId1, 100, 100,
OperationType.COORDINATING);
} else {
fireConcurrentAndParallelRequestsForUniformThroughPut(NUM_THREADS, shardIndexingPressure, shardId1, 100, 100,
OperationType.PRIMARY);
}
//Generating a load to such that the requests in the window shows degradation in throughput.
if (randomBoolean) {
fireAllThenCompleteConcurrentRequestsWithUniformDelay(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.get(settings),
shardIndexingPressure, shardId1, 100, 200, OperationType.COORDINATING);
} else {
fireAllThenCompleteConcurrentRequestsWithUniformDelay(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.get(settings),
shardIndexingPressure, shardId1, 100, 200, OperationType.PRIMARY);
}
//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);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId1 = new ShardId(index, 0);
//Generating a load to have a fair throughput
fireConcurrentAndParallelRequestsForUniformThroughPut(NUM_THREADS, shardIndexingPressure, shardId1, 100, 100,
OperationType.REPLICA);
//Generating a load to such that the requests in the window shows degradation in throughput.
fireAllThenCompleteConcurrentRequestsWithUniformDelay(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.get(settings),
shardIndexingPressure, shardId1, 100, 200, OperationType.REPLICA);
//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(), "20ms")
.build();
final int NUM_THREADS = scaledRandomIntBetween(110, 150);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.
Releasable[] releasables;
if (randomBoolean) {
releasables = fireConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 10, OperationType.COORDINATING);
} else {
releasables = fireConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 10, OperationType.PRIMARY);
}
//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(), "20ms")
.build();
final int NUM_THREADS = scaledRandomIntBetween(110, 150);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.
final Releasable[] releasables = fireConcurrentRequests(NUM_THREADS, shardIndexingPressure, shardId1, 10, OperationType.REPLICA);
//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(), "20ms")
.build();
final int NUM_THREADS = scaledRandomIntBetween(100, 150);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.
Releasable[] releasables;
if (randomBoolean) {
releasables = fireConcurrentRequestsWithUniformDelay(NUM_THREADS, shardIndexingPressure, shardId1, 10,
randomIntBetween(50, 100), OperationType.COORDINATING);
} else {
releasables = fireConcurrentRequestsWithUniformDelay(NUM_THREADS, shardIndexingPressure, shardId1, 10,
randomIntBetween(50, 100), OperationType.PRIMARY);
}
//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(), "20ms")
.build();
final int NUM_THREADS = scaledRandomIntBetween(100, 150);
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId1 = new ShardId(index, 0);
//Generating a load to such that the requests in the window shows degradation in throughput.
final Releasable[] releasables = fireConcurrentRequestsWithUniformDelay(NUM_THREADS, shardIndexingPressure, shardId1, 10,
randomIntBetween(50, 100), OperationType.COORDINATING);
//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(1, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1)
.getReplicaNodeLimitsBreachedRejections());
assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1)
.getReplicaLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(384, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId1).getCurrentReplicaLimits());
}
private void fireAndCompleteConcurrentRequests(int concurrency, ShardIndexingPressure shardIndexingPressure, ShardId shardId,
long bytes, OperationType operationType) throws Exception {
fireAndCompleteConcurrentRequestsWithUniformDelay(concurrency, shardIndexingPressure, shardId, bytes, randomIntBetween(5, 15),
operationType);
}
private void fireAndCompleteConcurrentRequestsWithUniformDelay(int concurrency, ShardIndexingPressure shardIndexingPressure,
ShardId shardId, long bytes, long delay,
OperationType operationType) throws Exception {
final Thread[] threads = new Thread[concurrency];
for (int i = 0; i < concurrency; i++) {
threads[i] = new Thread(() -> {
if(operationType == OperationType.COORDINATING) {
Releasable coordinating = shardIndexingPressure.markCoordinatingOperationStarted(shardId, bytes, false);
coordinating.close();
} else if (operationType == OperationType.PRIMARY){
Releasable primary = shardIndexingPressure.markPrimaryOperationStarted(shardId, bytes, false);
primary.close();
} else {
Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, bytes, false);
replica.close();
}
});
try {
Thread.sleep(delay);
} catch (InterruptedException e) {
//Do Nothing
}
threads[i].start();
}
for (Thread t : threads) {
t.join();
}
}
private Releasable[] fireConcurrentRequests(int concurrency, ShardIndexingPressure shardIndexingPressure, ShardId shardId,
long bytes, OperationType operationType) throws Exception {
return fireConcurrentRequestsWithUniformDelay(concurrency, shardIndexingPressure, shardId, bytes, 0, operationType);
}
private Releasable[] fireConcurrentRequestsWithUniformDelay(int concurrency, ShardIndexingPressure shardIndexingPressure,
ShardId shardId, long bytes, long delay,
OperationType operationType) throws Exception {
final Thread[] threads = new Thread[concurrency];
final Releasable[] releasable = new Releasable[concurrency];
for (int i = 0; i < concurrency; i++) {
int counter = i;
threads[i] = new Thread(() -> {
if(operationType == OperationType.COORDINATING) {
releasable[counter] = shardIndexingPressure.markCoordinatingOperationStarted(shardId, bytes, false);
} else if (operationType == OperationType.PRIMARY){
releasable[counter] = shardIndexingPressure.markPrimaryOperationStarted(shardId, bytes, false);
} else {
releasable[counter] = shardIndexingPressure.markReplicaOperationStarted(shardId, bytes, false);
}
try {
Thread.sleep(delay);
} catch (Exception e) {
//Do Nothing
}
});
threads[i].start();
}
for (Thread t : threads) {
t.join();
}
return releasable;
}
private void fireAllThenCompleteConcurrentRequests(int concurrency, ShardIndexingPressure shardIndexingPressure, ShardId shardId,
long bytes, OperationType operationType) throws Exception {
fireAllThenCompleteConcurrentRequestsWithUniformDelay(concurrency, shardIndexingPressure, shardId, bytes, 0, operationType);
}
private void fireAllThenCompleteConcurrentRequestsWithUniformDelay(int concurrency, ShardIndexingPressure shardIndexingPressure,
ShardId shardId, long bytes, long delay,
OperationType operationType) throws Exception {
final Releasable[] releasable = fireConcurrentRequestsWithUniformDelay(concurrency, shardIndexingPressure, shardId, bytes, delay,
operationType);
for (int i = 0; i < concurrency; i++) {
releasable[i].close();
}
}
private void fireConcurrentAndParallelRequestsForUniformThroughPut(int concurrency, ShardIndexingPressure shardIndexingPressure,
ShardId shardId, long bytes, long delay,
OperationType operationType) throws Exception {
final Thread[] threads = new Thread[concurrency];
for (int i = 0; i < concurrency; i++) {
threads[i] = new Thread(() -> {
for (int j = 0; j < randomIntBetween(400, 500); j++) {
Releasable releasable;
if(operationType == OperationType.COORDINATING) {
releasable = shardIndexingPressure.markCoordinatingOperationStarted(shardId, bytes, false);
} else if (operationType == OperationType.PRIMARY){
releasable = shardIndexingPressure.markPrimaryOperationStarted(shardId, bytes, false);
} else {
releasable = shardIndexingPressure.markReplicaOperationStarted(shardId, bytes, false);
}
try {
Thread.sleep(delay);
} catch (Exception e) {
//Do Nothing
}
releasable.close();
}
});
threads[i].start();
}
for (Thread t : threads) {
t.join();
}
}
}

View File

@ -0,0 +1,530 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Settings;
import org.opensearch.index.shard.ShardId;
import org.opensearch.test.OpenSearchTestCase;
import java.util.concurrent.TimeUnit;
public class ShardIndexingPressureMemoryManagerTests extends OpenSearchTestCase {
private final Settings settings = Settings.builder().put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10KB")
.put(ShardIndexingPressureMemoryManager.MAX_OUTSTANDING_REQUESTS.getKey(), 1)
.put(ShardIndexingPressureMemoryManager.SUCCESSFUL_REQUEST_ELAPSED_TIMEOUT.getKey(), "20ms")
.put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 2)
.build();
private final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
private final ShardIndexingPressureSettings shardIndexingPressureSettings =
new ShardIndexingPressureSettings(new ClusterService(settings, clusterSettings, null), settings,
IndexingPressure.MAX_INDEXING_BYTES.get(settings).getBytes());
private final Index index = new Index("IndexName", "UUID");
private final ShardId shardId1 = new ShardId(index, 0);
private final ShardId shardId2 = new ShardId(index, 1);
private final ShardIndexingPressureMemoryManager memoryManager = new ShardIndexingPressureMemoryManager(shardIndexingPressureSettings,
clusterSettings, settings);
public void testCoordinatingNodeLevelBreach() {
ShardIndexingPressureTracker tracker = memoryManager.getShardIndexingPressureTracker(shardId1);
assertFalse(memoryManager.isCoordinatingNodeLimitBreached(tracker, 1 * 1024));
assertTrue(memoryManager.isCoordinatingNodeLimitBreached(tracker, 11 * 1024));
}
public void testPrimaryNodeLevelBreach() {
ShardIndexingPressureTracker tracker = memoryManager.getShardIndexingPressureTracker(shardId1);
assertFalse(memoryManager.isPrimaryNodeLimitBreached(tracker, 1 * 1024));
assertTrue(memoryManager.isPrimaryNodeLimitBreached(tracker, 11 * 1024));
}
public void testReplicaNodeLevelBreach() {
ShardIndexingPressureTracker tracker = memoryManager.getShardIndexingPressureTracker(shardId1);
assertFalse(memoryManager.isReplicaNodeLimitBreached(tracker, 1 * 1024));
assertTrue(memoryManager.isReplicaNodeLimitBreached(tracker, 16 * 1024));
}
public void testCoordinatingPrimaryShardLimitsNotBreached() {
ShardIndexingPressureTracker tracker = memoryManager.getShardIndexingPressureTracker(shardId1);
tracker.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(1);
long requestStartTime = System.nanoTime();
assertFalse(memoryManager.isCoordinatingShardLimitBreached(tracker, 1 * 1024, requestStartTime));
assertFalse(memoryManager.isPrimaryShardLimitBreached(tracker, 1 * 1024, requestStartTime));
}
public void testReplicaShardLimitsNotBreached() {
ShardIndexingPressureTracker tracker = memoryManager.getShardIndexingPressureTracker(shardId1);
tracker.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(1);
long requestStartTime = System.nanoTime();
assertFalse(memoryManager.isReplicaShardLimitBreached(tracker, 1 * 1024, requestStartTime));
}
public void testCoordinatingPrimaryShardLimitsIncreasedAndSoftLimitNotBreached() {
ShardIndexingPressureTracker tracker = memoryManager.getShardIndexingPressureTracker(shardId1);
tracker.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(10);
long baseLimit = tracker.getPrimaryAndCoordinatingLimits();
long requestStartTime = System.nanoTime();
assertFalse(memoryManager.isCoordinatingShardLimitBreached(tracker, 1 * 1024, requestStartTime));
assertFalse(memoryManager.isPrimaryShardLimitBreached(tracker, 1 * 1024, requestStartTime));
assertTrue(tracker.getPrimaryAndCoordinatingLimits() > baseLimit);
assertEquals(tracker.getPrimaryAndCoordinatingLimits(), (long)(baseLimit/0.85));
}
public void testReplicaShardLimitsIncreasedAndSoftLimitNotBreached() {
ShardIndexingPressureTracker tracker = memoryManager.getShardIndexingPressureTracker(shardId1);
tracker.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(15);
long baseLimit = tracker.getReplicaLimits();
long requestStartTime = System.nanoTime();
assertFalse(memoryManager.isReplicaShardLimitBreached(tracker, 1 * 1024, requestStartTime));
assertTrue(tracker.getReplicaLimits() > baseLimit);
assertEquals(tracker.getReplicaLimits(), (long)(baseLimit/0.85));
}
public void testCoordinatingPrimarySoftLimitNotBreachedAndNodeLevelRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(4 * 1024);
tracker2.compareAndSetPrimaryAndCoordinatingLimits(tracker2.getPrimaryAndCoordinatingLimits(), 6 * 1024);
long limit1 = tracker1.getPrimaryAndCoordinatingLimits();
long limit2 = tracker2.getPrimaryAndCoordinatingLimits();
long requestStartTime = System.nanoTime();
assertTrue(memoryManager.isCoordinatingShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getCoordinatingOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getCoordinatingOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertTrue(memoryManager.isPrimaryShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getPrimaryOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getPrimaryOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(limit1, tracker1.getPrimaryAndCoordinatingLimits());
assertEquals(limit2, tracker2.getPrimaryAndCoordinatingLimits());
assertEquals(2, memoryManager.getTotalNodeLimitsBreachedRejections());
}
public void testReplicaShardLimitsSoftLimitNotBreachedAndNodeLevelRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(5 * 1024);
tracker2.compareAndSetReplicaLimits(tracker2.getReplicaLimits(), 10 * 1024);
long limit1 = tracker1.getReplicaLimits();
long limit2 = tracker2.getReplicaLimits();
long requestStartTime = System.nanoTime();
assertTrue(memoryManager.isReplicaShardLimitBreached(tracker1, 10 * 1024, requestStartTime));
assertEquals(limit1, tracker1.getReplicaLimits());
assertEquals(limit2, tracker2.getReplicaLimits());
assertEquals(1, tracker1.getReplicaOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getReplicaOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(1, memoryManager.getTotalNodeLimitsBreachedRejections());
}
public void testCoordinatingPrimarySoftLimitBreachedAndNodeLevelRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(4 * 1024);
tracker2.compareAndSetPrimaryAndCoordinatingLimits(tracker2.getPrimaryAndCoordinatingLimits(), 6 * 1024);
long limit1 = tracker1.getPrimaryAndCoordinatingLimits();
long limit2 = tracker2.getPrimaryAndCoordinatingLimits();
long requestStartTime = System.nanoTime();
assertTrue(memoryManager.isCoordinatingShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getCoordinatingOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getCoordinatingOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertTrue(memoryManager.isPrimaryShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getPrimaryOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getPrimaryOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(limit1, tracker1.getPrimaryAndCoordinatingLimits());
assertEquals(limit2, tracker2.getPrimaryAndCoordinatingLimits());
assertEquals(2, memoryManager.getTotalNodeLimitsBreachedRejections());
}
public void testReplicaShardLimitsSoftLimitBreachedAndNodeLevelRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(5 * 1024);
tracker2.compareAndSetReplicaLimits(tracker2.getReplicaLimits(), 12 * 1024);
long limit1 = tracker1.getReplicaLimits();
long limit2 = tracker2.getReplicaLimits();
long requestStartTime = System.nanoTime();
assertTrue(memoryManager.isReplicaShardLimitBreached(tracker1, 12 * 1024, requestStartTime));
assertEquals(limit1, tracker1.getReplicaLimits());
assertEquals(limit2, tracker2.getReplicaLimits());
assertEquals(1, tracker1.getReplicaOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getReplicaOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(1, memoryManager.getTotalNodeLimitsBreachedRejections());
}
public void testCoordinatingPrimarySoftLimitBreachedAndLastSuccessfulRequestLimitRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(4 * 1024);
tracker2.compareAndSetPrimaryAndCoordinatingLimits(tracker2.getPrimaryAndCoordinatingLimits(), 6 * 1024);
long limit1 = tracker1.getPrimaryAndCoordinatingLimits();
long limit2 = tracker2.getPrimaryAndCoordinatingLimits();
long requestStartTime = System.nanoTime();
long delay = TimeUnit.MILLISECONDS.toNanos(100);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().updateLastSuccessfulRequestTimestamp(requestStartTime - delay);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
assertTrue(memoryManager.isCoordinatingShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(0, tracker2.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
tracker1.getPrimaryOperationTracker().getPerformanceTracker().updateLastSuccessfulRequestTimestamp(requestStartTime - delay);
tracker1.getPrimaryOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getPrimaryOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
assertTrue(memoryManager.isPrimaryShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getPrimaryOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(0, tracker2.getPrimaryOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(limit1, tracker1.getPrimaryAndCoordinatingLimits());
assertEquals(limit2, tracker2.getPrimaryAndCoordinatingLimits());
assertEquals(2, memoryManager.getTotalLastSuccessfulRequestLimitsBreachedRejections());
}
public void testReplicaShardLimitsSoftLimitBreachedAndLastSuccessfulRequestLimitRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(5 * 1024);
tracker2.compareAndSetReplicaLimits(tracker2.getReplicaLimits(), 12 * 1024);
long limit1 = tracker1.getReplicaLimits();
long limit2 = tracker2.getReplicaLimits();
long requestStartTime = System.nanoTime();
long delay = TimeUnit.MILLISECONDS.toNanos(100);
tracker1.getReplicaOperationTracker().getPerformanceTracker().updateLastSuccessfulRequestTimestamp(requestStartTime - delay);
tracker1.getReplicaOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getReplicaOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
assertTrue(memoryManager.isReplicaShardLimitBreached(tracker1, 12 * 1024, requestStartTime));
assertEquals(limit1, tracker1.getReplicaLimits());
assertEquals(limit2, tracker2.getReplicaLimits());
assertEquals(1, tracker1.getReplicaOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(0, tracker2.getReplicaOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(1, memoryManager.getTotalLastSuccessfulRequestLimitsBreachedRejections());
}
public void testCoordinatingPrimarySoftLimitBreachedAndLessOutstandingRequestsAndNoLastSuccessfulRequestLimitRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(1 * 1024);
tracker2.compareAndSetPrimaryAndCoordinatingLimits(tracker2.getPrimaryAndCoordinatingLimits(), 6 * 1024);
long limit1 = tracker1.getPrimaryAndCoordinatingLimits();
long limit2 = tracker2.getPrimaryAndCoordinatingLimits();
long requestStartTime = System.nanoTime();
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().updateLastSuccessfulRequestTimestamp(requestStartTime - 100);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
assertFalse(memoryManager.isCoordinatingShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(0, tracker1.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(0, tracker2.getCoordinatingOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
tracker1.getPrimaryOperationTracker().getPerformanceTracker().updateLastSuccessfulRequestTimestamp(requestStartTime - 100);
tracker1.getPrimaryOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
assertFalse(memoryManager.isPrimaryShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(0, tracker1.getPrimaryOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(0, tracker2.getPrimaryOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertTrue(tracker1.getPrimaryAndCoordinatingLimits() > limit1);
assertEquals((long)(1 * 1024/0.85), tracker1.getPrimaryAndCoordinatingLimits());
assertEquals(limit2, tracker2.getPrimaryAndCoordinatingLimits());
assertEquals(0, memoryManager.getTotalLastSuccessfulRequestLimitsBreachedRejections());
}
public void testReplicaShardLimitsSoftLimitBreachedAndLessOutstandingRequestsAndNoLastSuccessfulRequestLimitRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(2 * 1024);
tracker2.compareAndSetReplicaLimits(tracker2.getReplicaLimits(), 12 * 1024);
long limit1 = tracker1.getReplicaLimits();
long limit2 = tracker2.getReplicaLimits();
long requestStartTime = System.nanoTime();
tracker1.getReplicaOperationTracker().getPerformanceTracker().updateLastSuccessfulRequestTimestamp(requestStartTime - 100);
tracker1.getReplicaOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
assertFalse(memoryManager.isReplicaShardLimitBreached(tracker1, 12 * 1024, requestStartTime));
assertTrue(tracker1.getReplicaLimits() > limit1);
assertEquals((long)(2 * 1024/0.85), tracker1.getReplicaLimits());
assertEquals(limit2, tracker2.getReplicaLimits());
assertEquals(0, tracker1.getReplicaOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(0, tracker2.getReplicaOperationTracker().getRejectionTracker()
.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(0, memoryManager.getTotalLastSuccessfulRequestLimitsBreachedRejections());
}
public void testCoordinatingPrimarySoftLimitBreachedAndThroughputDegradationLimitRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(4 * 1024);
tracker2.compareAndSetPrimaryAndCoordinatingLimits(tracker2.getPrimaryAndCoordinatingLimits(), 6 * 1024);
long limit1 = tracker1.getPrimaryAndCoordinatingLimits();
long limit2 = tracker2.getPrimaryAndCoordinatingLimits();
long requestStartTime = System.nanoTime();
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().updateThroughputMovingAverage(Double.doubleToLongBits(1d));
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getCoordinatingOperationTracker().getStatsTracker().incrementTotalBytes(60);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().addLatencyInMillis(10);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().addNewThroughout(1d);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().addNewThroughout(2d);
assertTrue(memoryManager.isCoordinatingShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getCoordinatingOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(0, tracker2.getCoordinatingOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
tracker1.getPrimaryOperationTracker().getPerformanceTracker().updateThroughputMovingAverage(Double.doubleToLongBits(1d));
tracker1.getPrimaryOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getPrimaryOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getPrimaryOperationTracker().getStatsTracker().incrementTotalBytes(60);
tracker1.getPrimaryOperationTracker().getPerformanceTracker().addLatencyInMillis(10);
tracker1.getPrimaryOperationTracker().getPerformanceTracker().addNewThroughout(1d);
tracker1.getPrimaryOperationTracker().getPerformanceTracker().addNewThroughout(2d);
assertTrue(memoryManager.isPrimaryShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getPrimaryOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(0, tracker2.getPrimaryOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(limit1, tracker1.getPrimaryAndCoordinatingLimits());
assertEquals(limit2, tracker2.getPrimaryAndCoordinatingLimits());
assertEquals(2, memoryManager.getTotalThroughputDegradationLimitsBreachedRejections());
}
public void testReplicaShardLimitsSoftLimitBreachedAndThroughputDegradationLimitRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(5 * 1024);
tracker2.compareAndSetReplicaLimits(tracker2.getReplicaLimits(), 12 * 1024);
long limit1 = tracker1.getReplicaLimits();
long limit2 = tracker2.getReplicaLimits();
long requestStartTime = System.nanoTime();
tracker1.getReplicaOperationTracker().getPerformanceTracker().updateThroughputMovingAverage(Double.doubleToLongBits(1d));
tracker1.getReplicaOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getReplicaOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getReplicaOperationTracker().getStatsTracker().incrementTotalBytes(80);
tracker1.getReplicaOperationTracker().getPerformanceTracker().addLatencyInMillis(10);
tracker1.getReplicaOperationTracker().getPerformanceTracker().addNewThroughout(1d);
tracker1.getReplicaOperationTracker().getPerformanceTracker().addNewThroughout(2d);
assertTrue(memoryManager.isReplicaShardLimitBreached(tracker1, 12 * 1024, requestStartTime));
assertEquals(limit1, tracker1.getReplicaLimits());
assertEquals(limit2, tracker2.getReplicaLimits());
assertEquals(1, tracker1.getReplicaOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(0, tracker2.getReplicaOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(1, memoryManager.getTotalThroughputDegradationLimitsBreachedRejections());
}
public void testCoordinatingPrimarySoftLimitBreachedAndMovingAverageQueueNotBuildUpAndNoThroughputDegradationLimitRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(1 * 1024);
tracker2.compareAndSetPrimaryAndCoordinatingLimits(tracker2.getPrimaryAndCoordinatingLimits(), 6 * 1024);
long limit1 = tracker1.getPrimaryAndCoordinatingLimits();
long limit2 = tracker2.getPrimaryAndCoordinatingLimits();
long requestStartTime = System.nanoTime();
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().updateThroughputMovingAverage
(Double.doubleToLongBits(1d));
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getCoordinatingOperationTracker().getStatsTracker().incrementTotalBytes(60);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().addLatencyInMillis(10);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().addNewThroughout(1d);
assertFalse(memoryManager.isCoordinatingShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(0, tracker1.getCoordinatingOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(0, tracker2.getCoordinatingOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(0, tracker1.getCoordinatingOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getCoordinatingOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
tracker1.getPrimaryOperationTracker().getPerformanceTracker().updateThroughputMovingAverage(Double.doubleToLongBits(1d));
tracker1.getPrimaryOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getPrimaryOperationTracker().getStatsTracker().incrementTotalBytes(60);
tracker1.getPrimaryOperationTracker().getPerformanceTracker().addLatencyInMillis(10);
tracker1.getPrimaryOperationTracker().getPerformanceTracker().addNewThroughout(1d);
assertFalse(memoryManager.isPrimaryShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(0, tracker1.getPrimaryOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(0, tracker2.getPrimaryOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(0, tracker1.getPrimaryOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getPrimaryOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertTrue(tracker1.getPrimaryAndCoordinatingLimits() > limit1);
assertEquals((long)(1 * 1024/0.85), tracker1.getPrimaryAndCoordinatingLimits());
assertEquals(limit2, tracker2.getPrimaryAndCoordinatingLimits());
assertEquals(0, memoryManager.getTotalThroughputDegradationLimitsBreachedRejections());
}
public void testReplicaShardLimitsSoftLimitBreachedAndMovingAverageQueueNotBuildUpAndNThroughputDegradationLimitRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(2 * 1024);
tracker2.compareAndSetReplicaLimits(tracker2.getReplicaLimits(), 12 * 1024);
long limit1 = tracker1.getReplicaLimits();
long limit2 = tracker2.getReplicaLimits();
long requestStartTime = System.nanoTime();
tracker1.getReplicaOperationTracker().getPerformanceTracker().updateThroughputMovingAverage(Double.doubleToLongBits(1d));
tracker1.getReplicaOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getReplicaOperationTracker().getStatsTracker().incrementTotalBytes(80);
tracker1.getReplicaOperationTracker().getPerformanceTracker().addLatencyInMillis(10);
tracker1.getReplicaOperationTracker().getPerformanceTracker().addNewThroughout(1d);
assertFalse(memoryManager.isReplicaShardLimitBreached(tracker1, 12 * 1024, requestStartTime));
assertTrue(tracker1.getReplicaLimits() > limit1);
assertEquals((long)(2 * 1024/0.85), tracker1.getReplicaLimits());
assertEquals(limit2, tracker2.getReplicaLimits());
assertEquals(0, tracker1.getReplicaOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(0, tracker2.getReplicaOperationTracker().getRejectionTracker()
.getThroughputDegradationLimitsBreachedRejections());
assertEquals(0, tracker1.getReplicaOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, memoryManager.getTotalThroughputDegradationLimitsBreachedRejections());
}
public void testCoordinatingPrimarySoftLimitBreachedAndNoSecondaryParameterBreachedAndNodeLevelRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(4 * 1024);
tracker2.compareAndSetPrimaryAndCoordinatingLimits(tracker2.getPrimaryAndCoordinatingLimits(), 6 * 1024);
long limit1 = tracker1.getPrimaryAndCoordinatingLimits();
long limit2 = tracker2.getPrimaryAndCoordinatingLimits();
long requestStartTime = System.nanoTime();
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().updateThroughputMovingAverage(Double.doubleToLongBits(1d));
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getCoordinatingOperationTracker().getStatsTracker().incrementTotalBytes(60);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().addLatencyInMillis(10);
tracker1.getCoordinatingOperationTracker().getPerformanceTracker().addNewThroughout(1d);
assertTrue(memoryManager.isCoordinatingShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getCoordinatingOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getCoordinatingOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
tracker1.getPrimaryOperationTracker().getPerformanceTracker().updateThroughputMovingAverage(Double.doubleToLongBits(1d));
tracker1.getPrimaryOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getPrimaryOperationTracker().getStatsTracker().incrementTotalBytes(60);
tracker1.getPrimaryOperationTracker().getPerformanceTracker().addLatencyInMillis(10);
tracker1.getPrimaryOperationTracker().getPerformanceTracker().addNewThroughout(1d);
assertTrue(memoryManager.isPrimaryShardLimitBreached(tracker1, 8 * 1024, requestStartTime));
assertEquals(1, tracker1.getPrimaryOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getPrimaryOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(limit1, tracker1.getPrimaryAndCoordinatingLimits());
assertEquals(limit2, tracker2.getPrimaryAndCoordinatingLimits());
assertEquals(2, memoryManager.getTotalNodeLimitsBreachedRejections());
}
public void testReplicaShardLimitsSoftLimitBreachedAndNoSecondaryParameterBreachedAndNodeLevelRejections() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
ShardIndexingPressureTracker tracker2 = memoryManager.getShardIndexingPressureTracker(shardId2);
tracker1.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(5 * 1024);
tracker2.compareAndSetReplicaLimits(tracker2.getReplicaLimits(), 12 * 1024);
long limit1 = tracker1.getReplicaLimits();
long limit2 = tracker2.getReplicaLimits();
long requestStartTime = System.nanoTime();
tracker1.getReplicaOperationTracker().getPerformanceTracker().updateThroughputMovingAverage(Double.doubleToLongBits(1d));
tracker1.getReplicaOperationTracker().getPerformanceTracker().incrementTotalOutstandingRequests();
tracker1.getReplicaOperationTracker().getStatsTracker().incrementTotalBytes(80);
tracker1.getReplicaOperationTracker().getPerformanceTracker().addLatencyInMillis(10);
tracker1.getReplicaOperationTracker().getPerformanceTracker().addNewThroughout(1d);
assertTrue(memoryManager.isReplicaShardLimitBreached(tracker1, 12 * 1024, requestStartTime));
assertEquals(limit1, tracker1.getReplicaLimits());
assertEquals(limit2, tracker2.getReplicaLimits());
assertEquals(1, tracker1.getReplicaOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(0, tracker2.getReplicaOperationTracker().getRejectionTracker().getNodeLimitsBreachedRejections());
assertEquals(1, memoryManager.getTotalNodeLimitsBreachedRejections());
}
public void testDecreaseShardPrimaryAndCoordinatingLimitsToBaseLimit() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
tracker1.compareAndSetPrimaryAndCoordinatingLimits(tracker1.getPrimaryAndCoordinatingLimits(), 1 * 1024);
tracker1.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(0);
long limit1 = tracker1.getPrimaryAndCoordinatingLimits();
memoryManager.decreaseShardPrimaryAndCoordinatingLimits(tracker1);
assertTrue(tracker1.getPrimaryAndCoordinatingLimits() < limit1);
assertEquals(10, tracker1.getPrimaryAndCoordinatingLimits());
}
public void testDecreaseShardReplicaLimitsToBaseLimit() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
tracker1.compareAndSetReplicaLimits(tracker1.getReplicaLimits(), 1 * 1024);
tracker1.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(0);
long limit1 = tracker1.getReplicaLimits();
memoryManager.decreaseShardReplicaLimits(tracker1);
assertTrue(tracker1.getReplicaLimits() < limit1);
assertEquals(15, tracker1.getReplicaLimits());
}
public void testDecreaseShardPrimaryAndCoordinatingLimits() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
tracker1.compareAndSetPrimaryAndCoordinatingLimits(tracker1.getPrimaryAndCoordinatingLimits(), 1 * 1024);
tracker1.getCommonOperationTracker().incrementCurrentCombinedCoordinatingAndPrimaryBytes(512);
long limit1 = tracker1.getPrimaryAndCoordinatingLimits();
memoryManager.decreaseShardPrimaryAndCoordinatingLimits(tracker1);
assertTrue(tracker1.getPrimaryAndCoordinatingLimits() < limit1);
assertEquals((long)(512/0.85), tracker1.getPrimaryAndCoordinatingLimits());
}
public void testDecreaseShardReplicaLimits() {
ShardIndexingPressureTracker tracker1 = memoryManager.getShardIndexingPressureTracker(shardId1);
tracker1.compareAndSetReplicaLimits(tracker1.getReplicaLimits(), 1 * 1024);
tracker1.getReplicaOperationTracker().getStatsTracker().incrementCurrentBytes(512);
long limit1 = tracker1.getReplicaLimits();
memoryManager.decreaseShardReplicaLimits(tracker1);
assertTrue(tracker1.getReplicaLimits() < limit1);
assertEquals((long)(512/0.85), tracker1.getReplicaLimits());
}
}

View File

@ -0,0 +1,80 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Settings;
import org.opensearch.test.OpenSearchTestCase;
public class ShardIndexingPressureSettingsTests extends OpenSearchTestCase {
private final Settings settings = Settings.builder()
.put(IndexingPressure.MAX_INDEXING_BYTES.getKey(), "10MB")
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), true)
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true)
.put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 2000)
.put(ShardIndexingPressureSettings.SHARD_MIN_LIMIT.getKey(), 0.001d)
.build();
final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
final ClusterService clusterService = new ClusterService(settings, clusterSettings, null);
public void testFromSettings() {
ShardIndexingPressureSettings shardIndexingPressureSettings = new ShardIndexingPressureSettings(clusterService, settings,
IndexingPressure.MAX_INDEXING_BYTES.get(settings).getBytes());
assertTrue(shardIndexingPressureSettings.isShardIndexingPressureEnabled());
assertTrue(shardIndexingPressureSettings.isShardIndexingPressureEnforced());
assertEquals(2000, shardIndexingPressureSettings.getRequestSizeWindow());
// Node level limits
long nodePrimaryAndCoordinatingLimits = shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits();
long tenMB = 10 * 1024 * 1024;
assertEquals(tenMB, nodePrimaryAndCoordinatingLimits);
assertEquals((long)(tenMB * 1.5), shardIndexingPressureSettings.getNodeReplicaLimits());
// Shard Level Limits
long shardPrimaryAndCoordinatingBaseLimits = (long) (nodePrimaryAndCoordinatingLimits * 0.001d);
assertEquals(shardPrimaryAndCoordinatingBaseLimits, shardIndexingPressureSettings.getShardPrimaryAndCoordinatingBaseLimits());
assertEquals((long)(shardPrimaryAndCoordinatingBaseLimits * 1.5),
shardIndexingPressureSettings.getShardReplicaBaseLimits());
}
public void testUpdateSettings() {
ShardIndexingPressureSettings shardIndexingPressureSettings = new ShardIndexingPressureSettings(clusterService, settings,
IndexingPressure.MAX_INDEXING_BYTES.get(settings).getBytes());
Settings.Builder updated = Settings.builder();
clusterSettings.updateDynamicSettings(Settings.builder()
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED.getKey(), false)
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), false)
.put(ShardIndexingPressureSettings.REQUEST_SIZE_WINDOW.getKey(), 4000)
.put(ShardIndexingPressureSettings.SHARD_MIN_LIMIT.getKey(), 0.003d)
.build(),
Settings.builder().put(settings), updated, getTestClass().getName());
clusterSettings.applySettings(updated.build());
assertFalse(shardIndexingPressureSettings.isShardIndexingPressureEnabled());
assertFalse(shardIndexingPressureSettings.isShardIndexingPressureEnforced());
assertEquals(4000, shardIndexingPressureSettings.getRequestSizeWindow());
// Node level limits
long nodePrimaryAndCoordinatingLimits = shardIndexingPressureSettings.getNodePrimaryAndCoordinatingLimits();
long tenMB = 10 * 1024 * 1024;
assertEquals(tenMB, nodePrimaryAndCoordinatingLimits);
assertEquals((long)(tenMB * 1.5), shardIndexingPressureSettings.getNodeReplicaLimits());
// Shard Level Limits
long shardPrimaryAndCoordinatingBaseLimits = (long) (nodePrimaryAndCoordinatingLimits * 0.003d);
assertEquals(shardPrimaryAndCoordinatingBaseLimits, shardIndexingPressureSettings.getShardPrimaryAndCoordinatingBaseLimits());
assertEquals((long)(shardPrimaryAndCoordinatingBaseLimits * 1.5),
shardIndexingPressureSettings.getShardReplicaBaseLimits());
}
}

View File

@ -0,0 +1,190 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index;
import org.junit.Before;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Settings;
import org.opensearch.index.shard.ShardId;
import org.opensearch.test.OpenSearchTestCase;
import java.util.Map;
public class ShardIndexingPressureStoreTests extends OpenSearchTestCase {
private final Settings settings = Settings.builder()
.put(ShardIndexingPressureStore.MAX_COLD_STORE_SIZE.getKey(), 200)
.build();
private final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
private final ShardIndexingPressureSettings shardIndexingPressureSettings =
new ShardIndexingPressureSettings(new ClusterService(settings, clusterSettings, null), settings,
IndexingPressure.MAX_INDEXING_BYTES.get(settings).getBytes());
private ShardIndexingPressureStore store;
private ShardId testShardId;
@Before
public void beforeTest() {
store = new ShardIndexingPressureStore(shardIndexingPressureSettings, clusterSettings, settings);
testShardId = new ShardId("index", "uuid", 0);
}
public void testShardIndexingPressureStoreGet() {
ShardIndexingPressureTracker tracker1 = store.getShardIndexingPressureTracker(testShardId);
ShardIndexingPressureTracker tracker2 = store.getShardIndexingPressureTracker(testShardId);
assertEquals(tracker1, tracker2);
}
public void testGetVerifyTrackerInHotStore() {
ShardIndexingPressureTracker tracker = store.getShardIndexingPressureTracker(testShardId);
Map<ShardId, ShardIndexingPressureTracker> hostStoreTrackers = store.getShardIndexingPressureHotStore();
assertEquals(1, hostStoreTrackers.size());
ShardIndexingPressureTracker hotStoreTracker = hostStoreTrackers.get(testShardId);
assertEquals(tracker, hotStoreTracker);
}
public void testTrackerCleanupFromHotStore() {
ShardIndexingPressureTracker tracker = store.getShardIndexingPressureTracker(testShardId);
Map<ShardId, ShardIndexingPressureTracker> hostStoreTrackers = store.getShardIndexingPressureHotStore();
assertEquals(1, hostStoreTrackers.size());
store.tryTrackerCleanupFromHotStore(tracker, () -> true);
hostStoreTrackers = store.getShardIndexingPressureHotStore();
assertEquals(0, hostStoreTrackers.size());
Map<ShardId, ShardIndexingPressureTracker> coldStoreTrackers = store.getShardIndexingPressureColdStore();
assertEquals(1, coldStoreTrackers.size());
ShardIndexingPressureTracker coldStoreTracker = coldStoreTrackers.get(testShardId);
assertEquals(tracker, coldStoreTracker);
}
public void testTrackerCleanupSkippedFromHotStore() {
ShardIndexingPressureTracker tracker = store.getShardIndexingPressureTracker(testShardId);
Map<ShardId, ShardIndexingPressureTracker> hostStoreTrackers = store.getShardIndexingPressureHotStore();
assertEquals(1, hostStoreTrackers.size());
store.tryTrackerCleanupFromHotStore(tracker, () -> false);
hostStoreTrackers = store.getShardIndexingPressureHotStore();
assertEquals(1, hostStoreTrackers.size());
ShardIndexingPressureTracker coldStoreTracker = hostStoreTrackers.get(testShardId);
assertEquals(tracker, coldStoreTracker);
}
public void testTrackerRestoredToHotStorePostCleanup() {
ShardIndexingPressureTracker tracker1 = store.getShardIndexingPressureTracker(testShardId);
Map<ShardId, ShardIndexingPressureTracker> hostStoreTrackers = store.getShardIndexingPressureHotStore();
assertEquals(1, hostStoreTrackers.size());
store.tryTrackerCleanupFromHotStore(tracker1, () -> true);
hostStoreTrackers = store.getShardIndexingPressureHotStore();
assertEquals(0, hostStoreTrackers.size());
ShardIndexingPressureTracker tracker2 = store.getShardIndexingPressureTracker(testShardId);
hostStoreTrackers = store.getShardIndexingPressureHotStore();
assertEquals(1, hostStoreTrackers.size());
assertEquals(tracker1, tracker2);
}
public void testTrackerEvictedFromColdStore() {
for (int i = 0; i <= ShardIndexingPressureStore.MAX_COLD_STORE_SIZE.get(settings); i++) {
ShardId shardId = new ShardId("index", "uuid", i);
ShardIndexingPressureTracker tracker = store.getShardIndexingPressureTracker(shardId);
store.tryTrackerCleanupFromHotStore(tracker, () -> true);
assertEquals(i + 1, store.getShardIndexingPressureColdStore().size());
}
// Verify cold store size is maximum
assertEquals(ShardIndexingPressureStore.MAX_COLD_STORE_SIZE.get(settings) + 1,
store.getShardIndexingPressureColdStore().size());
// get and remove one more tracker object from hot store
ShardId shardId = new ShardId("index", "uuid",
ShardIndexingPressureStore.MAX_COLD_STORE_SIZE.get(settings) + 1);
ShardIndexingPressureTracker tracker = store.getShardIndexingPressureTracker(shardId);
store.tryTrackerCleanupFromHotStore(tracker, () -> true);
// Verify all trackers objects purged from cold store except the last
assertEquals(1, store.getShardIndexingPressureColdStore().size());
assertEquals(tracker, store.getShardIndexingPressureColdStore().get(shardId));
}
public void testShardIndexingPressureStoreConcurrentGet() throws Exception {
ShardIndexingPressureTracker tracker = store.getShardIndexingPressureTracker(testShardId);
final int NUM_THREADS = scaledRandomIntBetween(100, 500);
final Thread[] threads = new Thread[NUM_THREADS];
final ShardIndexingPressureTracker[] trackers = new ShardIndexingPressureTracker[NUM_THREADS];
for (int i = 0; i < NUM_THREADS; i++) {
int counter = i;
threads[i] = new Thread(() -> {
trackers[counter] = store.getShardIndexingPressureTracker(testShardId);
});
threads[i].start();
}
for (Thread t : threads) {
t.join();
}
for (int i = 0; i < NUM_THREADS; i++) {
assertEquals(tracker, trackers[i]);
}
assertEquals(1, store.getShardIndexingPressureHotStore().size());
assertEquals(1, store.getShardIndexingPressureColdStore().size());
assertEquals(tracker, store.getShardIndexingPressureHotStore().get(testShardId));
assertEquals(tracker, store.getShardIndexingPressureColdStore().get(testShardId));
}
public void testShardIndexingPressureStoreConcurrentGetAndCleanup() throws Exception {
ShardIndexingPressureTracker tracker = store.getShardIndexingPressureTracker(testShardId);
final int NUM_THREADS = scaledRandomIntBetween(100, 500);
final Thread[] threads = new Thread[NUM_THREADS];
for (int i = 0; i < NUM_THREADS; i++) {
threads[i] = new Thread(() -> {
ShardIndexingPressureTracker tracker1 = store.getShardIndexingPressureTracker(testShardId);
assertEquals(tracker, tracker1);
store.tryTrackerCleanupFromHotStore(tracker, () -> true);
});
threads[i].start();
}
for (Thread t : threads) {
t.join();
}
assertEquals(0, store.getShardIndexingPressureHotStore().size());
assertEquals(1, store.getShardIndexingPressureColdStore().size());
assertEquals(tracker, store.getShardIndexingPressureColdStore().get(testShardId));
}
public void testTrackerConcurrentEvictionFromColdStore() throws Exception {
int maxColdStoreSize = ShardIndexingPressureStore.MAX_COLD_STORE_SIZE.get(settings);
final int NUM_THREADS = scaledRandomIntBetween(maxColdStoreSize * 2, maxColdStoreSize * 8);
final Thread[] threads = new Thread[NUM_THREADS];
for (int i = 0; i < NUM_THREADS; i++) {
int counter = i;
threads[i] = new Thread(() -> {
ShardId shardId = new ShardId("index", "uuid", counter);
ShardIndexingPressureTracker tracker = store.getShardIndexingPressureTracker(shardId);
store.tryTrackerCleanupFromHotStore(tracker, () -> true);
});
threads[i].start();
}
for (Thread t : threads) {
t.join();
}
assertEquals(0, store.getShardIndexingPressureHotStore().size());
assertTrue(store.getShardIndexingPressureColdStore().size() <= maxColdStoreSize + 1);
}
}

View File

@ -0,0 +1,820 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.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(), "20ms")
.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() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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 = shardIndexingPressure.stats();
assertEquals(60, nodeStats.getCurrentCoordinatingBytes());
assertEquals(20, nodeStats.getCurrentPrimaryBytes());
assertEquals(80, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(35, nodeStats.getCurrentReplicaBytes());
IndexingPressurePerShardStats shardStats = shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId);
assertEquals(60, shardStats.getCurrentCoordinatingBytes());
assertEquals(20, shardStats.getCurrentPrimaryBytes());
assertEquals(80, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(35, shardStats.getCurrentReplicaBytes());
}
IndexingPressureStats nodeStats = shardIndexingPressure.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.shardStats().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() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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 = shardIndexingPressure.stats();
assertEquals(10, nodeStats.getCurrentCoordinatingBytes());
assertEquals(15, nodeStats.getCurrentPrimaryBytes());
assertEquals(10, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes());
IndexingPressurePerShardStats shardStats = shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId);
assertEquals(10, shardStats.getCurrentCoordinatingBytes());
assertEquals(15, shardStats.getCurrentPrimaryBytes());
assertEquals(10, shardStats.getCurrentCombinedCoordinatingAndPrimaryBytes());
}
IndexingPressureStats nodeStats = shardIndexingPressure.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.shardStats().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() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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 = shardIndexingPressure.stats();
assertEquals(1, nodeStats.getCoordinatingRejections());
assertEquals(1024 * 6, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes());
IndexingPressurePerShardStats shardStats = shardIndexingPressure.shardStats().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 = shardIndexingPressure.stats();
assertEquals(1, nodeStats.getPrimaryRejections());
assertEquals(1024 * 6, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes());
IndexingPressurePerShardStats shardStats = shardIndexingPressure.shardStats()
.getIndexingPressureShardStats(shardId);
assertEquals(1, shardStats.getPrimaryRejections());
assertEquals(1024 * 6, nodeStats.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(1, shardStats.getPrimaryNodeLimitsBreachedRejections());
}
long preForceRejections = shardIndexingPressure.stats().getPrimaryRejections();
long preForcedShardRejections = shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getPrimaryRejections();
// Primary can be forced
Releasable forced = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1024 * 2, true);
assertEquals(preForceRejections, shardIndexingPressure.stats().getPrimaryRejections());
assertEquals(1024 * 8, shardIndexingPressure.stats().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(preForcedShardRejections, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getPrimaryRejections());
assertEquals(1024 * 8, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(preForcedShardRejections, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getPrimaryNodeLimitsBreachedRejections());
forced.close();
// Local to coordinating node primary actions not rejected
IndexingPressureStats preLocalNodeStats = shardIndexingPressure.stats();
IndexingPressurePerShardStats preLocalShardStats = shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId);
Releasable local = shardIndexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(shardId, 1024 * 2);
assertEquals(preLocalNodeStats.getPrimaryRejections(), shardIndexingPressure.stats().getPrimaryRejections());
assertEquals(1024 * 6, shardIndexingPressure.stats().getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(preLocalNodeStats.getCurrentPrimaryBytes() + 1024 * 2, shardIndexingPressure.stats().getCurrentPrimaryBytes());
assertEquals(preLocalShardStats.getPrimaryRejections(), shardIndexingPressure.shardStats()
.getIndexingPressureShardStats(shardId).getPrimaryRejections());
assertEquals(1024 * 6, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(preLocalShardStats.getCurrentPrimaryBytes() + 1024 * 2, shardIndexingPressure.shardStats()
.getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes());
assertEquals(preLocalShardStats.getPrimaryNodeLimitsBreachedRejections(), shardIndexingPressure.shardStats()
.getIndexingPressureShardStats(shardId).getPrimaryNodeLimitsBreachedRejections());
local.close();
}
assertEquals(1024 * 8, shardIndexingPressure.stats().getTotalCombinedCoordinatingAndPrimaryBytes());
assertNull(shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId));
assertEquals(1024 * 8, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId)
.getTotalCombinedCoordinatingAndPrimaryBytes());
}
public void testReplicaRejections() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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, shardIndexingPressure.stats().getCurrentReplicaBytes());
assertEquals(1024 * 12, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes());
// Replica will be rejected once we cross 15KB Shard Limit
expectThrows(OpenSearchRejectedExecutionException.class, () -> shardIndexingPressure
.markReplicaOperationStarted(shardId, 1024 * 2, false));
IndexingPressureStats nodeStats = shardIndexingPressure.stats();
assertEquals(1, nodeStats.getReplicaRejections());
assertEquals(1024 * 12, nodeStats.getCurrentReplicaBytes());
IndexingPressurePerShardStats shardStats = shardIndexingPressure.shardStats().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, shardIndexingPressure.stats().getReplicaRejections());
assertEquals(1024 * 14, shardIndexingPressure.stats().getCurrentReplicaBytes());
assertEquals(1, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getReplicaRejections());
assertEquals(1024 * 14, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes());
assertEquals(1, shardStats.getReplicaNodeLimitsBreachedRejections());
forced.close();
replica2.close();
}
assertEquals(1024 * 14, shardIndexingPressure.stats().getTotalReplicaBytes());
assertNull(shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId));
assertEquals(1024 * 14, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId).getTotalReplicaBytes());
}
public void testCoordinatingPrimaryShardLimitIncrease() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes());
assertEquals(4, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(10, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentPrimaryAndCoordinatingLimits()); // Base Limit
if (randomBoolean) {
Releasable coordinating1 = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 6, false);
assertEquals(8, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes());
assertEquals(10, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(11, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentPrimaryAndCoordinatingLimits()); // Increased Limit
coordinating1.close();
} else {
Releasable primary1 = shardIndexingPressure.markPrimaryOperationStarted(shardId, 6, false);
assertEquals(8, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes());
assertEquals(10, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(11, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentPrimaryAndCoordinatingLimits()); // Increased Limit
primary1.close();
}
}
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
try (Releasable replica = shardIndexingPressure.markReplicaOperationStarted(shardId, 2, false)) {
assertEquals(2, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes());
assertEquals(15, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentReplicaLimits()); // Base Limit
Releasable replica1 = shardIndexingPressure.markReplicaOperationStarted(shardId, 14, false);
assertEquals(16, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes());
assertEquals(18, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentReplicaLimits()); // Increased Limit
replica1.close();
}
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCoordinatingBytes());
assertEquals(4 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentPrimaryBytes());
assertEquals(8 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals((long)(8*1024/0.85), shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentPrimaryAndCoordinatingLimits());
}
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes());
assertEquals((long)(11 * 1024/0.85), shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentReplicaLimits());
}
IndexingPressurePerShardStats shardStoreStats = shardIndexingPressure.shardStats().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(), "20ms")
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true)
.build();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes());
assertEquals(1 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes());
assertEquals(2 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals((long)(2*1024/0.85), shardIndexingPressure.shardStats().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 = shardIndexingPressure.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(), "20ms")
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), true)
.build();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes());
assertEquals((long)(1*1024/0.85), shardIndexingPressure.shardStats().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 = shardIndexingPressure.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(), "20ms")
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), false)
.build();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes());
assertEquals(1 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes());
assertEquals(2 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals((long)(2*1024/0.85), shardIndexingPressure.shardStats().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 = shardIndexingPressure.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(), "20ms")
.put(ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENFORCED.getKey(), false)
.build();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes());
assertEquals((long)(1*1024/0.85), shardIndexingPressure.shardStats().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 = shardIndexingPressure.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();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes());
assertEquals(4 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes());
assertEquals(8 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals((long)(8*1024/0.85), shardIndexingPressure.shardStats().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 = shardIndexingPressure.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();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes());
assertEquals((long)(4*1024/0.85), shardIndexingPressure.shardStats().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 = shardIndexingPressure.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();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes());
assertEquals(4 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId).getCurrentPrimaryBytes());
assertEquals(8 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals((long)(8*1024/0.85), shardIndexingPressure.shardStats().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 = shardIndexingPressure.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();
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId).getCurrentReplicaBytes());
assertEquals((long)(4*1024/0.85), shardIndexingPressure.shardStats().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 = shardIndexingPressure.stats();
assertEquals(0, nodeStats.getReplicaRejections());
assertEquals(0, nodeStats.getCurrentReplicaBytes());
}
public void testShardLimitIncreaseMultipleShards() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentCoordinatingBytes());
assertEquals(4 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals((long)(4 * 1024 / 0.85), shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId1)
.getCurrentPrimaryAndCoordinatingLimits());
assertEquals(4 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId2)
.getCurrentCoordinatingBytes());
assertEquals(4 * 1024, shardIndexingPressure.shardStats().getIndexingPressureShardStats(shardId2)
.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals((long)(4 * 1024 / 0.85), shardIndexingPressure.shardStats().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() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
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.shardStats().getIndexingPressureShardStats(shardId)
.getCurrentCoordinatingBytes());
}
assertEquals(0, shardIndexingPressure.coldStats().getIndexingPressureShardStats(shardId).getCurrentCoordinatingBytes());
}
public void testAssertionOnReleaseExecutedTwice() {
ShardIndexingPressure shardIndexingPressure = new ShardIndexingPressure(settings, clusterService);
Index index = new Index("IndexName", "UUID");
ShardId shardId = new ShardId(index, 0);
String assertionMessage = "ShardIndexingPressure Release is called twice";
Releasable releasable = shardIndexingPressure.markCoordinatingOperationStarted(shardId, 1024, false);
releasable.close();
expectThrows(AssertionError.class, assertionMessage, releasable::close);
releasable = shardIndexingPressure.markPrimaryOperationLocalToCoordinatingNodeStarted(shardId, 1024);
releasable.close();
expectThrows(AssertionError.class, assertionMessage, releasable::close);
releasable = shardIndexingPressure.markPrimaryOperationStarted(shardId, 1024, false);
releasable.close();
expectThrows(AssertionError.class, assertionMessage, releasable::close);
releasable = shardIndexingPressure.markReplicaOperationStarted(shardId, 1024, false);
releasable.close();
expectThrows(AssertionError.class, assertionMessage, releasable::close);
}
}

View File

@ -0,0 +1,73 @@
/*
* SPDX-License-Identifier: Apache-2.0
*
* The OpenSearch Contributors require contributions made to
* this file be licensed under the Apache-2.0 license or a
* compatible open source license.
*/
package org.opensearch.index;
import org.opensearch.index.ShardIndexingPressureTracker.OperationTracker;
import org.opensearch.index.ShardIndexingPressureTracker.CommonOperationTracker;
import org.opensearch.index.ShardIndexingPressureTracker.StatsTracker;
import org.opensearch.index.ShardIndexingPressureTracker.RejectionTracker;
import org.opensearch.index.ShardIndexingPressureTracker.PerformanceTracker;
import org.opensearch.index.shard.ShardId;
import org.opensearch.test.OpenSearchTestCase;
public class ShardIndexingPressureTrackerTests extends OpenSearchTestCase {
public void testShardIndexingPressureTracker() {
ShardId shardId = new ShardId("index", "uuid", 0);
ShardIndexingPressureTracker shardIndexingPressureTracker = new ShardIndexingPressureTracker(
shardId, 100L, 150L);
assertEquals(shardId, shardIndexingPressureTracker.getShardId());
assertEquals(100L, shardIndexingPressureTracker.getPrimaryAndCoordinatingLimits());
assertEquals(150L, shardIndexingPressureTracker.getReplicaLimits());
OperationTracker coordinatingTracker = shardIndexingPressureTracker.getCoordinatingOperationTracker();
assertStatsTracker(coordinatingTracker.getStatsTracker());
assertRejectionTracker(coordinatingTracker.getRejectionTracker());
assertPerformanceTracker(coordinatingTracker.getPerformanceTracker());
OperationTracker primaryTracker = shardIndexingPressureTracker.getPrimaryOperationTracker();
assertStatsTracker(primaryTracker.getStatsTracker());
assertRejectionTracker(primaryTracker.getRejectionTracker());
assertPerformanceTracker(primaryTracker.getPerformanceTracker());
OperationTracker replicaTracker = shardIndexingPressureTracker.getReplicaOperationTracker();
assertStatsTracker(replicaTracker.getStatsTracker());
assertRejectionTracker(replicaTracker.getRejectionTracker());
assertPerformanceTracker(replicaTracker.getPerformanceTracker());
CommonOperationTracker commonOperationTracker = shardIndexingPressureTracker.getCommonOperationTracker();
assertEquals(0L, commonOperationTracker.getCurrentCombinedCoordinatingAndPrimaryBytes());
assertEquals(0L, commonOperationTracker.getTotalCombinedCoordinatingAndPrimaryBytes());
}
private void assertStatsTracker(StatsTracker statsTracker) {
assertEquals(0L, statsTracker.getCurrentBytes());
assertEquals(0L, statsTracker.getTotalBytes());
assertEquals(0L, statsTracker.getRequestCount());
}
private void assertRejectionTracker(RejectionTracker rejectionTracker) {
assertEquals(0L, rejectionTracker.getTotalRejections());
assertEquals(0L, rejectionTracker.getNodeLimitsBreachedRejections());
assertEquals(0L, rejectionTracker.getLastSuccessfulRequestLimitsBreachedRejections());
assertEquals(0L, rejectionTracker.getThroughputDegradationLimitsBreachedRejections());
}
private void assertPerformanceTracker(PerformanceTracker performanceTracker) {
assertEquals(0L, performanceTracker.getLatencyInMillis());
assertEquals(0L, performanceTracker.getLastSuccessfulRequestTimestamp());
assertEquals(0L, performanceTracker.getTotalOutstandingRequests());
assertEquals(0L, performanceTracker.getThroughputMovingAverage());
assertTrue(performanceTracker.addNewThroughout(10.0));
assertEquals(1L, performanceTracker.getThroughputMovingQueueSize());
assertEquals(10.0, performanceTracker.getFirstThroughput(), 0.0);
assertEquals(0L, performanceTracker.getThroughputMovingQueueSize());
}
}

View File

@ -33,7 +33,6 @@
package org.opensearch.index.seqno;
import org.opensearch.action.ActionListener;
import org.opensearch.index.IndexingPressure;
import org.opensearch.action.support.ActionFilters;
import org.opensearch.action.support.ActionTestUtils;
import org.opensearch.action.support.PlainActionFuture;
@ -44,6 +43,7 @@ import org.opensearch.common.settings.Settings;
import org.opensearch.core.internal.io.IOUtils;
import org.opensearch.index.Index;
import org.opensearch.index.IndexService;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.shard.ShardId;
import org.opensearch.indices.IndicesService;
@ -120,7 +120,7 @@ public class RetentionLeaseSyncActionTests extends OpenSearchTestCase {
threadPool,
shardStateAction,
new ActionFilters(Collections.emptySet()),
new IndexingPressure(Settings.EMPTY),
new IndexingPressureService(Settings.EMPTY, clusterService),
new SystemIndices(emptyMap()));
final RetentionLeases retentionLeases = mock(RetentionLeases.class);
final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases);
@ -158,7 +158,7 @@ public class RetentionLeaseSyncActionTests extends OpenSearchTestCase {
threadPool,
shardStateAction,
new ActionFilters(Collections.emptySet()),
new IndexingPressure(Settings.EMPTY),
new IndexingPressureService(Settings.EMPTY, clusterService),
new SystemIndices(emptyMap()));
final RetentionLeases retentionLeases = mock(RetentionLeases.class);
final RetentionLeaseSyncAction.Request request = new RetentionLeaseSyncAction.Request(indexShard.shardId(), retentionLeases);
@ -199,7 +199,7 @@ public class RetentionLeaseSyncActionTests extends OpenSearchTestCase {
threadPool,
shardStateAction,
new ActionFilters(Collections.emptySet()),
new IndexingPressure(Settings.EMPTY),
new IndexingPressureService(Settings.EMPTY, clusterService),
new SystemIndices(emptyMap()));
assertNull(action.indexBlockLevel());

View File

@ -166,7 +166,7 @@ import org.opensearch.env.TestEnvironment;
import org.opensearch.gateway.MetaStateService;
import org.opensearch.gateway.TransportNodesListGatewayStartedShards;
import org.opensearch.index.Index;
import org.opensearch.index.IndexingPressure;
import org.opensearch.index.IndexingPressureService;
import org.opensearch.index.analysis.AnalysisRegistry;
import org.opensearch.index.seqno.GlobalCheckpointSyncAction;
import org.opensearch.index.seqno.RetentionLeaseSyncer;
@ -1572,7 +1572,7 @@ public class SnapshotResiliencyTests extends OpenSearchTestCase {
threadPool,
shardStateAction,
actionFilters,
new IndexingPressure(settings),
new IndexingPressureService(settings, clusterService),
new SystemIndices(emptyMap()))),
new GlobalCheckpointSyncAction(
settings,
@ -1599,7 +1599,7 @@ public class SnapshotResiliencyTests extends OpenSearchTestCase {
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 IndexingPressureService(settings, clusterService), new SystemIndices(emptyMap()));
actions.put(BulkAction.INSTANCE,
new TransportBulkAction(threadPool, transportService, clusterService,
new IngestService(
@ -1608,7 +1608,7 @@ public class SnapshotResiliencyTests extends OpenSearchTestCase {
Collections.emptyList(), client),
transportShardBulkAction, client, actionFilters, indexNameExpressionResolver,
new AutoCreateIndex(settings, clusterSettings, indexNameExpressionResolver, new SystemIndices(emptyMap())),
new IndexingPressure(settings),
new IndexingPressureService(settings, clusterService),
new SystemIndices(emptyMap())
));
final RestoreService restoreService = new RestoreService(

View File

@ -99,7 +99,7 @@ public class MockInternalClusterInfoService extends InternalClusterInfoService {
.toArray(FsInfo.Path[]::new)), nodeStats.getTransport(),
nodeStats.getHttp(), nodeStats.getBreaker(), nodeStats.getScriptStats(), nodeStats.getDiscoveryStats(),
nodeStats.getIngestStats(), nodeStats.getAdaptiveSelectionStats(), nodeStats.getScriptCacheStats(),
nodeStats.getIndexingPressureStats());
nodeStats.getIndexingPressureStats(), nodeStats.getShardIndexingPressureStats());
}).collect(Collectors.toList());
}

View File

@ -2546,7 +2546,7 @@ public final class InternalTestCluster extends TestCluster {
NodeService nodeService = getInstanceFromNode(NodeService.class, nodeAndClient.node);
CommonStatsFlags flags = new CommonStatsFlags(Flag.FieldData, Flag.QueryCache, Flag.Segments);
NodeStats stats = nodeService.stats(flags,
false, false, false, false, false, false, false, false, false, false, false, false, false, false);
false, false, false, false, false, false, false, false, false, false, false, false, false, false, false);
assertThat("Fielddata size must be 0 on node: " + stats.getNode(),
stats.getIndices().getFieldData().getMemorySizeInBytes(), equalTo(0L));
assertThat("Query cache size must be 0 on node: " + stats.getNode(),