Skip to content

Commit

Permalink
Cluster manager task throttling feature [Final PR] (opensearch-projec…
Browse files Browse the repository at this point in the history
…t#4986)

Basic Throttler Framework / Exponential Basic back off policy.
Add basic thorttler/exponential backoff policy for retry/Defination o… opensearch-project#3527
Changes required in Master node to perform throttling.
Master node changes for master task throttling opensearch-project#3882
Changes required in Data node to perform retry on throttling.
Data node changes for master task throttling opensearch-project#4204
Provide support for all task type in throttling framework.
Onboarding of few task types to throttling opensearch-project#4542
Integration Tests (Fix timeout exception and Add Integ test for Master task throttling opensearch-project#4588

Signed-off-by: Dhwanil Patel <dhwanip@amazon.com>
  • Loading branch information
dhwanilpatel committed Nov 4, 2022
1 parent d3e95d6 commit 0fb0835
Show file tree
Hide file tree
Showing 49 changed files with 2,059 additions and 93 deletions.
7 changes: 7 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,13 @@ Inspired from [Keep a Changelog](https://keepachangelog.com/en/1.0.0/)
- Added resource usage trackers for in-flight cancellation of SearchShardTask ([#4805](https://github.com/opensearch-project/OpenSearch/pull/4805))
- Added search backpressure stats API ([#4932](https://github.com/opensearch-project/OpenSearch/pull/4932))
- Added changes for graceful node decommission ([#4586](https://github.com/opensearch-project/OpenSearch/pull/4586))
- Add Cluster manager task throttling framework. Cluster manager node will throttle task submission based on throttling thresholds.
This throttling will be at task type level. Data nodes will perform retries on these throttling exception with exponential delay. (PR: [#4986](https://github.com/opensearch-project/OpenSearch/pull/4986)) ( Issue : [#479](https://github.com/opensearch-project/OpenSearch/issues/479))
- Throttling Exception / New Backoff policy([#3527](https://github.com/opensearch-project/OpenSearch/pull/3527))
- Cluster Manager node side change([#3882](https://github.com/opensearch-project/OpenSearch/pull/3882))
- Data node side change([#4204](https://github.com/opensearch-project/OpenSearch/pull/4204))
- on-boarding of tasks([#4542](https://github.com/opensearch-project/OpenSearch/pull/4542))
- Integs ([4588](https://github.com/opensearch-project/OpenSearch/pull/4588))

### Dependencies
- Bumps `com.diffplug.spotless` from 6.9.1 to 6.10.0
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,214 @@
/*
* 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.clustermanager;

import org.opensearch.action.ActionListener;
import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest;
import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest;
import org.opensearch.cluster.metadata.ProcessClusterEventTimeoutException;
import org.opensearch.cluster.service.ClusterManagerThrottlingException;
import org.opensearch.common.settings.Settings;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.transport.TransportService;
import org.opensearch.transport.TransportMessageListener;

import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;

import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked;

@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.SUITE, numDataNodes = 0)
public class ClusterManagerTaskThrottlingIT extends OpenSearchIntegTestCase {

/*
* This integ test will test end-end cluster manager throttling feature for
* remote cluster manager.
*
* It will check the number of request coming to cluster manager node
* should be total number of requests + throttled requests from cluster manager.
* This will ensure the end-end feature is working as cluster manager is throwing
* Throttling exception and data node is performing retries on it.
*
*/
public void testThrottlingForRemoteClusterManager() throws Exception {
try {
internalCluster().beforeTest(random());
String clusterManagerNode = internalCluster().startClusterManagerOnlyNode();
String dataNode = internalCluster().startDataOnlyNode();
int throttlingLimit = randomIntBetween(1, 5);
createIndex("test");
setPutMappingThrottlingLimit(throttlingLimit);

TransportService clusterManagerTransportService = (internalCluster().getInstance(TransportService.class, clusterManagerNode));
AtomicInteger requestCountOnClusterManager = new AtomicInteger();
AtomicInteger throttledRequest = new AtomicInteger();
int totalRequest = randomIntBetween(throttlingLimit, 5 * throttlingLimit);
CountDownLatch latch = new CountDownLatch(totalRequest);

clusterManagerTransportService.addMessageListener(new TransportMessageListener() {
@Override
public void onRequestReceived(long requestId, String action) {
if (action.contains("mapping")) {
requestCountOnClusterManager.incrementAndGet();
}
}

@Override
public void onResponseSent(long requestId, String action, Exception error) {
if (action.contains("mapping")) {
throttledRequest.incrementAndGet();
assertEquals(ClusterManagerThrottlingException.class, error.getClass());
}
}
});

ActionListener listener = new ActionListener() {
@Override
public void onResponse(Object o) {
latch.countDown();
}

@Override
public void onFailure(Exception e) {
latch.countDown();
throw new AssertionError(e);
}
};

executePutMappingRequests(totalRequest, dataNode, listener);
latch.await();

assertEquals(totalRequest + throttledRequest.get(), requestCountOnClusterManager.get());
assertBusy(
() -> { assertEquals(clusterService().getMasterService().numberOfThrottledPendingTasks(), throttledRequest.get()); }
);
} finally {
clusterSettingCleanUp();
}
}

/*
* This will test the throttling feature for single node.
*
* Here we will assert the client behaviour that client's request is not
* failed, i.e. Throttling exception is not passed to the client.
* Data node will internally do the retry and request should pass.
*
*/
public void testThrottlingForSingleNode() throws Exception {
try {
internalCluster().beforeTest(random());
String node = internalCluster().startNode();
int throttlingLimit = randomIntBetween(1, 5);
createIndex("test");
setPutMappingThrottlingLimit(throttlingLimit);

AtomicInteger successfulRequest = new AtomicInteger();
int totalRequest = randomIntBetween(throttlingLimit, 3 * throttlingLimit);
CountDownLatch latch = new CountDownLatch(totalRequest);

ActionListener listener = new ActionListener() {
@Override
public void onResponse(Object o) {
latch.countDown();
successfulRequest.incrementAndGet();
}

@Override
public void onFailure(Exception e) {
latch.countDown();
throw new AssertionError(e);
}
};
executePutMappingRequests(totalRequest, node, listener);

latch.await();
assertEquals(totalRequest, successfulRequest.get());
} finally {
clusterSettingCleanUp();
}
}

/*
* This will test the timeout of tasks during throttling.
*
* Here we will assert the client behaviour that client's request is not
* failed with throttling exception but timeout exception.
* It also verifies that if limit is set to 0, all tasks are getting timedout.
*/

public void testTimeoutWhileThrottling() throws Exception {
try {
internalCluster().beforeTest(random());
String node = internalCluster().startNode();
int throttlingLimit = 0; // throttle all the tasks
createIndex("test");
setPutMappingThrottlingLimit(throttlingLimit);

AtomicInteger timedoutRequest = new AtomicInteger();
int totalRequest = randomIntBetween(1, 5);
CountDownLatch latch = new CountDownLatch(totalRequest);

ActionListener listener = new ActionListener() {
@Override
public void onResponse(Object o) {
latch.countDown();
throw new AssertionError("Request should not succeed");
}

@Override
public void onFailure(Exception e) {
latch.countDown();
assertTrue(e instanceof ProcessClusterEventTimeoutException);
timedoutRequest.incrementAndGet();
}
};
executePutMappingRequests(totalRequest, node, listener);

latch.await();
assertEquals(totalRequest, timedoutRequest.get()); // verifying all requests were timed out with 0 throttling limit
} finally {
clusterSettingCleanUp();
}
}

private void executePutMappingRequests(int totalRequest, String node, ActionListener listener) throws Exception {
Thread[] threads = new Thread[totalRequest];
for (int i = 0; i < totalRequest; i++) {
PutMappingRequest putMappingRequest = new PutMappingRequest("test").source("field" + i, "type=text");
threads[i] = new Thread(new Runnable() {
@Override
public void run() {
internalCluster().client(node).admin().indices().putMapping(putMappingRequest, listener);
}
});
}
for (int i = 0; i < totalRequest; i++) {
threads[i].run();
}
for (int i = 0; i < totalRequest; i++) {
threads[i].join();
}
}

private void setPutMappingThrottlingLimit(int throttlingLimit) {
ClusterUpdateSettingsRequest settingsRequest = new ClusterUpdateSettingsRequest();
Settings settings = Settings.builder().put("cluster_manager.throttling.thresholds.put-mapping.value", throttlingLimit).build();
settingsRequest.transientSettings(settings);
assertAcked(client().admin().cluster().updateSettings(settingsRequest).actionGet());
}

private void clusterSettingCleanUp() {
// We need to remove the throttling limit from setting as part of test cleanup
ClusterUpdateSettingsRequest settingsRequest = new ClusterUpdateSettingsRequest();
Settings settings = Settings.builder().put("cluster_manager.throttling.thresholds.put-mapping.value", (String) null).build();
settingsRequest.transientSettings(settings);
assertAcked(client().admin().cluster().updateSettings(settingsRequest).actionGet());
}
}
9 changes: 8 additions & 1 deletion server/src/main/java/org/opensearch/OpenSearchException.java
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@

import org.opensearch.action.support.replication.ReplicationOperation;
import org.opensearch.cluster.action.shard.ShardStateAction;
import org.opensearch.cluster.service.ClusterManagerThrottlingException;
import org.opensearch.common.CheckedFunction;
import org.opensearch.common.Nullable;
import org.opensearch.common.ParseField;
Expand Down Expand Up @@ -1620,7 +1621,13 @@ private enum OpenSearchExceptionHandle {
org.opensearch.cluster.decommission.NodeDecommissionedException.class,
org.opensearch.cluster.decommission.NodeDecommissionedException::new,
164,
V_2_4_0
V_3_0_0
),
CLUSTER_MANAGER_TASK_THROTTLED_EXCEPTION(
ClusterManagerThrottlingException.class,
ClusterManagerThrottlingException::new,
165,
Version.V_2_4_0
);

final Class<? extends OpenSearchException> exceptionClass;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,8 @@
import org.opensearch.cluster.routing.allocation.command.AbstractAllocateAllocationCommand;
import org.opensearch.cluster.routing.allocation.command.AllocateStalePrimaryAllocationCommand;
import org.opensearch.cluster.routing.allocation.command.AllocationCommand;
import org.opensearch.cluster.service.ClusterManagerTaskKeys;
import org.opensearch.cluster.service.ClusterManagerTaskThrottler;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.Priority;
import org.opensearch.common.Strings;
Expand All @@ -80,6 +82,7 @@ public class TransportClusterRerouteAction extends TransportClusterManagerNodeAc
private static final Logger logger = LogManager.getLogger(TransportClusterRerouteAction.class);

private final AllocationService allocationService;
private static ClusterManagerTaskThrottler.ThrottlingKey clusterRerouteTaskKey;

@Inject
public TransportClusterRerouteAction(
Expand All @@ -100,6 +103,8 @@ public TransportClusterRerouteAction(
indexNameExpressionResolver
);
this.allocationService = allocationService;
// Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction.
clusterRerouteTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.CLUSTER_REROUTE_API_KEY, true);
}

@Override
Expand Down Expand Up @@ -241,6 +246,11 @@ static class ClusterRerouteResponseAckedClusterStateUpdateTask extends AckedClus
this.allocationService = allocationService;
}

@Override
public ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey() {
return clusterRerouteTaskKey;
}

@Override
protected ClusterRerouteResponse newResponse(boolean acknowledged) {
return new ClusterRerouteResponse(acknowledged, clusterStateToSend, explanations);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,8 @@
import org.opensearch.cluster.metadata.Metadata;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.cluster.routing.allocation.AllocationService;
import org.opensearch.cluster.service.ClusterManagerTaskKeys;
import org.opensearch.cluster.service.ClusterManagerTaskThrottler;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.Nullable;
import org.opensearch.common.Priority;
Expand All @@ -73,6 +75,8 @@ public class TransportClusterUpdateSettingsAction extends TransportClusterManage

private final ClusterSettings clusterSettings;

private final ClusterManagerTaskThrottler.ThrottlingKey clusterUpdateSettingTaskKey;

@Inject
public TransportClusterUpdateSettingsAction(
TransportService transportService,
Expand All @@ -95,6 +99,10 @@ public TransportClusterUpdateSettingsAction(
);
this.allocationService = allocationService;
this.clusterSettings = clusterSettings;

// Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction.
clusterUpdateSettingTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.CLUSTER_UPDATE_SETTINGS_KEY, true);

}

@Override
Expand Down Expand Up @@ -136,6 +144,11 @@ protected void clusterManagerOperation(

private volatile boolean changed = false;

@Override
public ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey() {
return clusterUpdateSettingTaskKey;
}

@Override
protected ClusterUpdateSettingsResponse newResponse(boolean acknowledged) {
return new ClusterUpdateSettingsResponse(acknowledged, updater.getTransientUpdates(), updater.getPersistentUpdate());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@
import org.opensearch.cluster.block.ClusterBlockException;
import org.opensearch.cluster.block.ClusterBlockLevel;
import org.opensearch.cluster.metadata.IndexNameExpressionResolver;
import org.opensearch.cluster.service.ClusterManagerTaskKeys;
import org.opensearch.cluster.service.ClusterManagerTaskThrottler;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.inject.Inject;
import org.opensearch.common.io.stream.StreamInput;
Expand All @@ -57,6 +59,7 @@
public class TransportDeleteStoredScriptAction extends TransportClusterManagerNodeAction<DeleteStoredScriptRequest, AcknowledgedResponse> {

private final ScriptService scriptService;
private final ClusterManagerTaskThrottler.ThrottlingKey deleteScriptTaskKey;

@Inject
public TransportDeleteStoredScriptAction(
Expand All @@ -77,6 +80,8 @@ public TransportDeleteStoredScriptAction(
indexNameExpressionResolver
);
this.scriptService = scriptService;
// Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction.
deleteScriptTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.DELETE_SCRIPT_KEY, true);
}

@Override
Expand All @@ -95,7 +100,7 @@ protected void clusterManagerOperation(
ClusterState state,
ActionListener<AcknowledgedResponse> listener
) throws Exception {
scriptService.deleteStoredScript(clusterService, request, listener);
scriptService.deleteStoredScript(clusterService, request, deleteScriptTaskKey, listener);
}

@Override
Expand Down
Loading

0 comments on commit 0fb0835

Please sign in to comment.