diff --git a/CHANGELOG.md b/CHANGELOG.md
index de0f602807790..8602960b254c2 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -27,6 +27,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),
 - Adding access to noSubMatches and noOverlappingMatches in Hyphenation ([#13895](https://github.com/opensearch-project/OpenSearch/pull/13895))
 - Star tree mapping changes ([#14605](https://github.com/opensearch-project/OpenSearch/pull/14605))
 - Add support for index level max slice count setting for concurrent segment search ([#15336](https://github.com/opensearch-project/OpenSearch/pull/15336))
+- Support cancellation for cat shards and node stats API.([#13966](https://github.com/opensearch-project/OpenSearch/pull/13966))
 - [Streaming Indexing] Introduce bulk HTTP API streaming flavor ([#15381](https://github.com/opensearch-project/OpenSearch/pull/15381))
 - Add concurrent search support for Derived Fields ([#15326](https://github.com/opensearch-project/OpenSearch/pull/15326))
 - [Workload Management] Add query group stats constructs ([#15343](https://github.com/opensearch-project/OpenSearch/pull/15343)))
diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsActionIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsActionIT.java
new file mode 100644
index 0000000000000..b86521dedf739
--- /dev/null
+++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsActionIT.java
@@ -0,0 +1,129 @@
+/*
+ * 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.action.admin.cluster.shards;
+
+import org.opensearch.action.admin.cluster.state.ClusterStateResponse;
+import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
+import org.opensearch.cluster.metadata.IndexMetadata;
+import org.opensearch.cluster.routing.ShardRouting;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.core.action.ActionListener;
+import org.opensearch.core.common.Strings;
+import org.opensearch.core.tasks.TaskCancelledException;
+import org.opensearch.test.InternalTestCluster;
+import org.opensearch.test.OpenSearchIntegTestCase;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.opensearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING;
+import static org.opensearch.common.unit.TimeValue.timeValueMillis;
+import static org.opensearch.search.SearchService.NO_TIMEOUT;
+
+@OpenSearchIntegTestCase.ClusterScope(numDataNodes = 0, scope = OpenSearchIntegTestCase.Scope.TEST)
+public class TransportCatShardsActionIT extends OpenSearchIntegTestCase {
+
+    public void testCatShardsWithSuccessResponse() throws InterruptedException {
+        internalCluster().startClusterManagerOnlyNodes(1);
+        List<String> nodes = internalCluster().startDataOnlyNodes(3);
+        createIndex(
+            "test",
+            Settings.builder()
+                .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
+                .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2)
+                .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "60m")
+                .build()
+        );
+        ensureGreen("test");
+
+        final CatShardsRequest shardsRequest = new CatShardsRequest();
+        shardsRequest.setCancelAfterTimeInterval(NO_TIMEOUT);
+        shardsRequest.setIndices(Strings.EMPTY_ARRAY);
+        CountDownLatch latch = new CountDownLatch(1);
+        client().execute(CatShardsAction.INSTANCE, shardsRequest, new ActionListener<CatShardsResponse>() {
+            @Override
+            public void onResponse(CatShardsResponse catShardsResponse) {
+                ClusterStateResponse clusterStateResponse = catShardsResponse.getClusterStateResponse();
+                IndicesStatsResponse indicesStatsResponse = catShardsResponse.getIndicesStatsResponse();
+                for (ShardRouting shard : clusterStateResponse.getState().routingTable().allShards()) {
+                    assertEquals("test", shard.getIndexName());
+                    assertNotNull(indicesStatsResponse.asMap().get(shard));
+                }
+                latch.countDown();
+            }
+
+            @Override
+            public void onFailure(Exception e) {
+                fail();
+                latch.countDown();
+            }
+        });
+        latch.await();
+    }
+
+    public void testCatShardsWithTimeoutException() throws IOException, AssertionError, InterruptedException {
+        List<String> masterNodes = internalCluster().startClusterManagerOnlyNodes(1);
+        List<String> nodes = internalCluster().startDataOnlyNodes(3);
+        createIndex(
+            "test",
+            Settings.builder()
+                .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
+                .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2)
+                .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "60m")
+                .build()
+        );
+        ensureGreen("test");
+
+        Settings clusterManagerDataPathSettings = internalCluster().dataPathSettings(masterNodes.get(0));
+        // Dropping master node to delay in cluster state call.
+        internalCluster().stopRandomNode(InternalTestCluster.nameFilter(masterNodes.get(0)));
+
+        CountDownLatch latch = new CountDownLatch(2);
+        new Thread(() -> {
+            try {
+                // Ensures the cancellation timeout expires.
+                Thread.sleep(2000);
+                // Starting master node to proceed in cluster state call.
+                internalCluster().startClusterManagerOnlyNode(
+                    Settings.builder().put("node.name", masterNodes.get(0)).put(clusterManagerDataPathSettings).build()
+                );
+                latch.countDown();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }).start();
+
+        final CatShardsRequest shardsRequest = new CatShardsRequest();
+        TimeValue timeoutInterval = timeValueMillis(1000);
+        shardsRequest.setCancelAfterTimeInterval(timeoutInterval);
+        shardsRequest.clusterManagerNodeTimeout(timeValueMillis(2500));
+        shardsRequest.setIndices(Strings.EMPTY_ARRAY);
+        client().execute(CatShardsAction.INSTANCE, shardsRequest, new ActionListener<CatShardsResponse>() {
+            @Override
+            public void onResponse(CatShardsResponse catShardsResponse) {
+                // onResponse should not be called.
+                latch.countDown();
+                throw new AssertionError(
+                    "The cat shards action is expected to fail with a TaskCancelledException, but it received a successful response instead."
+                );
+            }
+
+            @Override
+            public void onFailure(Exception e) {
+                assertSame(e.getClass(), TaskCancelledException.class);
+                assertEquals(e.getMessage(), "Cancellation timeout of " + timeoutInterval + " is expired");
+                latch.countDown();
+            }
+        });
+        latch.await();
+    }
+
+}
diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java
index 10fae3a4bcf2b..35def090c0fc5 100644
--- a/server/src/main/java/org/opensearch/action/ActionModule.java
+++ b/server/src/main/java/org/opensearch/action/ActionModule.java
@@ -85,7 +85,9 @@
 import org.opensearch.action.admin.cluster.reroute.TransportClusterRerouteAction;
 import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsAction;
 import org.opensearch.action.admin.cluster.settings.TransportClusterUpdateSettingsAction;
+import org.opensearch.action.admin.cluster.shards.CatShardsAction;
 import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsAction;
+import org.opensearch.action.admin.cluster.shards.TransportCatShardsAction;
 import org.opensearch.action.admin.cluster.shards.TransportClusterSearchShardsAction;
 import org.opensearch.action.admin.cluster.shards.routing.weighted.delete.ClusterDeleteWeightedRoutingAction;
 import org.opensearch.action.admin.cluster.shards.routing.weighted.delete.TransportDeleteWeightedRoutingAction;
@@ -639,6 +641,7 @@ public <Request extends ActionRequest, Response extends ActionResponse> void reg
         actions.register(ClusterGetWeightedRoutingAction.INSTANCE, TransportGetWeightedRoutingAction.class);
         actions.register(ClusterDeleteWeightedRoutingAction.INSTANCE, TransportDeleteWeightedRoutingAction.class);
         actions.register(IndicesStatsAction.INSTANCE, TransportIndicesStatsAction.class);
+        actions.register(CatShardsAction.INSTANCE, TransportCatShardsAction.class);
         actions.register(IndicesSegmentsAction.INSTANCE, TransportIndicesSegmentsAction.class);
         actions.register(IndicesShardStoresAction.INSTANCE, TransportIndicesShardStoresAction.class);
         actions.register(CreateIndexAction.INSTANCE, TransportCreateIndexAction.class);
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsAction.java
new file mode 100644
index 0000000000000..381a950fe9a79
--- /dev/null
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsAction.java
@@ -0,0 +1,25 @@
+/*
+ * 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.action.admin.cluster.shards;
+
+import org.opensearch.action.ActionType;
+
+/**
+ * Transport action for cat shards
+ *
+ * @opensearch.internal
+ */
+public class CatShardsAction extends ActionType<CatShardsResponse> {
+    public static final CatShardsAction INSTANCE = new CatShardsAction();
+    public static final String NAME = "cluster:monitor/shards";
+
+    private CatShardsAction() {
+        super(NAME, CatShardsResponse::new);
+    }
+}
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsRequest.java
new file mode 100644
index 0000000000000..49299777db8ae
--- /dev/null
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsRequest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.action.admin.cluster.shards;
+
+import org.opensearch.action.ActionRequestValidationException;
+import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.core.common.io.stream.StreamInput;
+import org.opensearch.core.tasks.TaskId;
+import org.opensearch.rest.action.admin.cluster.ClusterAdminTask;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * A request of _cat/shards.
+ *
+ * @opensearch.api
+ */
+public class CatShardsRequest extends ClusterManagerNodeReadRequest<CatShardsRequest> {
+
+    private String[] indices;
+    private TimeValue cancelAfterTimeInterval;
+
+    public CatShardsRequest() {}
+
+    public CatShardsRequest(StreamInput in) throws IOException {
+        super(in);
+    }
+
+    @Override
+    public ActionRequestValidationException validate() {
+        return null;
+    }
+
+    public void setIndices(String[] indices) {
+        this.indices = indices;
+    }
+
+    public String[] getIndices() {
+        return this.indices;
+    }
+
+    public void setCancelAfterTimeInterval(TimeValue timeout) {
+        this.cancelAfterTimeInterval = timeout;
+    }
+
+    public TimeValue getCancelAfterTimeInterval() {
+        return this.cancelAfterTimeInterval;
+    }
+
+    @Override
+    public ClusterAdminTask createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
+        return new ClusterAdminTask(id, type, action, parentTaskId, headers, this.cancelAfterTimeInterval);
+    }
+}
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsResponse.java
new file mode 100644
index 0000000000000..3dd88a2cda037
--- /dev/null
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/CatShardsResponse.java
@@ -0,0 +1,57 @@
+/*
+ * 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.action.admin.cluster.shards;
+
+import org.opensearch.action.admin.cluster.state.ClusterStateResponse;
+import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
+import org.opensearch.core.action.ActionResponse;
+import org.opensearch.core.common.io.stream.StreamInput;
+import org.opensearch.core.common.io.stream.StreamOutput;
+
+import java.io.IOException;
+
+/**
+ * A response of a cat shards request.
+ *
+ * @opensearch.api
+ */
+public class CatShardsResponse extends ActionResponse {
+
+    private ClusterStateResponse clusterStateResponse = null;
+
+    private IndicesStatsResponse indicesStatsResponse = null;
+
+    public CatShardsResponse() {}
+
+    public CatShardsResponse(StreamInput in) throws IOException {
+        super(in);
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        clusterStateResponse.writeTo(out);
+        indicesStatsResponse.writeTo(out);
+    }
+
+    public void setClusterStateResponse(ClusterStateResponse clusterStateResponse) {
+        this.clusterStateResponse = clusterStateResponse;
+    }
+
+    public ClusterStateResponse getClusterStateResponse() {
+        return this.clusterStateResponse;
+    }
+
+    public void setIndicesStatsResponse(IndicesStatsResponse indicesStatsResponse) {
+        this.indicesStatsResponse = indicesStatsResponse;
+    }
+
+    public IndicesStatsResponse getIndicesStatsResponse() {
+        return this.indicesStatsResponse;
+    }
+}
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsAction.java
new file mode 100644
index 0000000000000..224d3cbc5f10a
--- /dev/null
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportCatShardsAction.java
@@ -0,0 +1,110 @@
+/*
+ * 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.action.admin.cluster.shards;
+
+import org.opensearch.action.admin.cluster.state.ClusterStateRequest;
+import org.opensearch.action.admin.cluster.state.ClusterStateResponse;
+import org.opensearch.action.admin.indices.stats.IndicesStatsRequest;
+import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
+import org.opensearch.action.support.ActionFilters;
+import org.opensearch.action.support.HandledTransportAction;
+import org.opensearch.action.support.TimeoutTaskCancellationUtility;
+import org.opensearch.client.node.NodeClient;
+import org.opensearch.common.inject.Inject;
+import org.opensearch.core.action.ActionListener;
+import org.opensearch.core.action.NotifyOnceListener;
+import org.opensearch.tasks.CancellableTask;
+import org.opensearch.tasks.Task;
+import org.opensearch.transport.TransportService;
+
+/**
+ * Perform cat shards action
+ *
+ * @opensearch.internal
+ */
+public class TransportCatShardsAction extends HandledTransportAction<CatShardsRequest, CatShardsResponse> {
+
+    private final NodeClient client;
+
+    @Inject
+    public TransportCatShardsAction(NodeClient client, TransportService transportService, ActionFilters actionFilters) {
+        super(CatShardsAction.NAME, transportService, actionFilters, CatShardsRequest::new);
+        this.client = client;
+    }
+
+    @Override
+    public void doExecute(Task parentTask, CatShardsRequest shardsRequest, ActionListener<CatShardsResponse> listener) {
+        final ClusterStateRequest clusterStateRequest = new ClusterStateRequest();
+        clusterStateRequest.setShouldCancelOnTimeout(true);
+        clusterStateRequest.local(shardsRequest.local());
+        clusterStateRequest.clusterManagerNodeTimeout(shardsRequest.clusterManagerNodeTimeout());
+        clusterStateRequest.clear().nodes(true).routingTable(true).indices(shardsRequest.getIndices());
+        assert parentTask instanceof CancellableTask;
+        clusterStateRequest.setParentTask(client.getLocalNodeId(), parentTask.getId());
+
+        ActionListener<CatShardsResponse> originalListener = new NotifyOnceListener<CatShardsResponse>() {
+            @Override
+            protected void innerOnResponse(CatShardsResponse catShardsResponse) {
+                listener.onResponse(catShardsResponse);
+            }
+
+            @Override
+            protected void innerOnFailure(Exception e) {
+                listener.onFailure(e);
+            }
+        };
+        ActionListener<CatShardsResponse> cancellableListener = TimeoutTaskCancellationUtility.wrapWithCancellationListener(
+            client,
+            (CancellableTask) parentTask,
+            ((CancellableTask) parentTask).getCancellationTimeout(),
+            originalListener,
+            e -> {
+                originalListener.onFailure(e);
+            }
+        );
+        CatShardsResponse catShardsResponse = new CatShardsResponse();
+        try {
+            client.admin().cluster().state(clusterStateRequest, new ActionListener<ClusterStateResponse>() {
+                @Override
+                public void onResponse(ClusterStateResponse clusterStateResponse) {
+                    catShardsResponse.setClusterStateResponse(clusterStateResponse);
+                    IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest();
+                    indicesStatsRequest.setShouldCancelOnTimeout(true);
+                    indicesStatsRequest.all();
+                    indicesStatsRequest.indices(shardsRequest.getIndices());
+                    indicesStatsRequest.setParentTask(client.getLocalNodeId(), parentTask.getId());
+                    try {
+                        client.admin().indices().stats(indicesStatsRequest, new ActionListener<IndicesStatsResponse>() {
+                            @Override
+                            public void onResponse(IndicesStatsResponse indicesStatsResponse) {
+                                catShardsResponse.setIndicesStatsResponse(indicesStatsResponse);
+                                cancellableListener.onResponse(catShardsResponse);
+                            }
+
+                            @Override
+                            public void onFailure(Exception e) {
+                                cancellableListener.onFailure(e);
+                            }
+                        });
+                    } catch (Exception e) {
+                        cancellableListener.onFailure(e);
+                    }
+                }
+
+                @Override
+                public void onFailure(Exception e) {
+                    cancellableListener.onFailure(e);
+                }
+            });
+        } catch (Exception e) {
+            cancellableListener.onFailure(e);
+        }
+
+    }
+}
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java
index 90a52f7406d57..66bb64ca5580b 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java
@@ -41,8 +41,12 @@
 import org.opensearch.core.common.Strings;
 import org.opensearch.core.common.io.stream.StreamInput;
 import org.opensearch.core.common.io.stream.StreamOutput;
+import org.opensearch.core.tasks.TaskId;
+import org.opensearch.rest.action.admin.cluster.ClusterAdminTask;
+import org.opensearch.tasks.Task;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Transport request for obtaining cluster state
@@ -211,4 +215,13 @@ public ClusterStateRequest waitForMetadataVersion(long waitForMetadataVersion) {
         this.waitForMetadataVersion = waitForMetadataVersion;
         return this;
     }
+
+    @Override
+    public Task createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
+        if (this.getShouldCancelOnTimeout()) {
+            return new ClusterAdminTask(id, type, action, parentTaskId, headers);
+        } else {
+            return super.createTask(id, type, action, parentTaskId, headers);
+        }
+    }
 }
diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsRequest.java
index 2b64464a76899..c36e53098d166 100644
--- a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsRequest.java
+++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsRequest.java
@@ -36,8 +36,12 @@
 import org.opensearch.common.annotation.PublicApi;
 import org.opensearch.core.common.io.stream.StreamInput;
 import org.opensearch.core.common.io.stream.StreamOutput;
+import org.opensearch.core.tasks.TaskId;
+import org.opensearch.rest.action.admin.cluster.ClusterAdminTask;
+import org.opensearch.tasks.Task;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * A request to get indices level stats. Allow to enable different stats to be returned.
@@ -103,6 +107,15 @@ public IndicesStatsRequest groups(String... groups) {
         return this;
     }
 
+    @Override
+    public Task createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
+        if (this.getShouldCancelOnTimeout()) {
+            return new ClusterAdminTask(id, type, action, parentTaskId, headers);
+        } else {
+            return super.createTask(id, type, action, parentTaskId, headers);
+        }
+    }
+
     public String[] groups() {
         return this.flags.groups();
     }
diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java
index 6242081cd2371..900a886481fe6 100644
--- a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java
+++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java
@@ -69,7 +69,7 @@ public class IndicesStatsResponse extends BroadcastResponse {
         shards = in.readArray(ShardStats::new, (size) -> new ShardStats[size]);
     }
 
-    IndicesStatsResponse(
+    public IndicesStatsResponse(
         ShardStats[] shards,
         int totalShards,
         int successfulShards,
diff --git a/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java b/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java
index 88bf7ebea8e52..8c4927afa9a14 100644
--- a/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java
+++ b/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java
@@ -312,8 +312,9 @@ protected void doExecute(Task task, SearchRequest searchRequest, ActionListener<
             listener = TimeoutTaskCancellationUtility.wrapWithCancellationListener(
                 client,
                 (CancellableTask) task,
-                clusterService.getClusterSettings(),
-                listener
+                clusterService.getClusterSettings().get(SEARCH_CANCEL_AFTER_TIME_INTERVAL_SETTING),
+                listener,
+                e -> {}
             );
         }
         executeRequest(task, searchRequest, this::searchAsyncAction, listener);
diff --git a/server/src/main/java/org/opensearch/action/support/TimeoutTaskCancellationUtility.java b/server/src/main/java/org/opensearch/action/support/TimeoutTaskCancellationUtility.java
index a317a45eab31f..d2e7121a592cb 100644
--- a/server/src/main/java/org/opensearch/action/support/TimeoutTaskCancellationUtility.java
+++ b/server/src/main/java/org/opensearch/action/support/TimeoutTaskCancellationUtility.java
@@ -12,11 +12,12 @@
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.message.ParameterizedMessage;
 import org.opensearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
+import org.opensearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
 import org.opensearch.client.OriginSettingClient;
 import org.opensearch.client.node.NodeClient;
-import org.opensearch.common.settings.ClusterSettings;
 import org.opensearch.common.unit.TimeValue;
 import org.opensearch.core.action.ActionListener;
+import org.opensearch.core.tasks.TaskCancelledException;
 import org.opensearch.core.tasks.TaskId;
 import org.opensearch.search.SearchService;
 import org.opensearch.tasks.CancellableTask;
@@ -27,7 +28,6 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.opensearch.action.admin.cluster.node.tasks.get.GetTaskAction.TASKS_ORIGIN;
-import static org.opensearch.action.search.TransportSearchAction.SEARCH_CANCEL_AFTER_TIME_INTERVAL_SETTING;
 
 /**
  * Utility to cancel a timeout task
@@ -38,25 +38,25 @@ public class TimeoutTaskCancellationUtility {
 
     private static final Logger logger = LogManager.getLogger(TimeoutTaskCancellationUtility.class);
 
+    private static final AtomicBoolean executeResponseOrFailureOnce = new AtomicBoolean(true);
+
     /**
      * Wraps a listener with a timeout listener {@link TimeoutRunnableListener} to schedule the task cancellation for provided tasks on
      * generic thread pool
      * @param client - {@link NodeClient}
      * @param taskToCancel - task to schedule cancellation for
-     * @param clusterSettings - {@link ClusterSettings}
+     * @param timeout - {@link TimeValue}
      * @param listener - original listener associated with the task
      * @return wrapped listener
      */
     public static <Response> ActionListener<Response> wrapWithCancellationListener(
         NodeClient client,
         CancellableTask taskToCancel,
-        ClusterSettings clusterSettings,
-        ActionListener<Response> listener
+        TimeValue timeout,
+        ActionListener<Response> listener,
+        TimeoutHandler timeoutHandler
     ) {
-        final TimeValue globalTimeout = clusterSettings.get(SEARCH_CANCEL_AFTER_TIME_INTERVAL_SETTING);
-        final TimeValue timeoutInterval = (taskToCancel.getCancellationTimeout() == null)
-            ? globalTimeout
-            : taskToCancel.getCancellationTimeout();
+        final TimeValue timeoutInterval = (taskToCancel.getCancellationTimeout() == null) ? timeout : taskToCancel.getCancellationTimeout();
         // Note: -1 (or no timeout) will help to turn off cancellation. The combinations will be request level set at -1 or request level
         // set to null and cluster level set to -1.
         ActionListener<Response> listenerToReturn = listener;
@@ -72,24 +72,30 @@ public static <Response> ActionListener<Response> wrapWithCancellationListener(
                 // force the origin to execute the cancellation as a system user
                 new OriginSettingClient(client, TASKS_ORIGIN).admin()
                     .cluster()
-                    .cancelTasks(
-                        cancelTasksRequest,
-                        ActionListener.wrap(
-                            r -> logger.debug(
+                    .cancelTasks(cancelTasksRequest, new ActionListener<CancelTasksResponse>() {
+                        @Override
+                        public void onResponse(CancelTasksResponse cancelTasksResponse) {
+                            logger.debug(
                                 "Scheduled cancel task with timeout: {} for original task: {} is successfully completed",
                                 timeoutInterval,
                                 cancelTasksRequest.getTaskId()
-                            ),
-                            e -> logger.error(
+                            );
+                            // Notify the timeoutHandler that the task was canceled due to timeout
+                            timeoutHandler.onTimeout(new TaskCancelledException(cancelTasksRequest.getReason()));
+                        }
+
+                        @Override
+                        public void onFailure(Exception e) {
+                            logger.error(
                                 new ParameterizedMessage(
                                     "Scheduled cancel task with timeout: {} for original task: {} is failed",
                                     timeoutInterval,
                                     cancelTasksRequest.getTaskId()
                                 ),
                                 e
-                            )
-                        )
-                    );
+                            );
+                        }
+                    });
             });
             wrappedListener.cancellable = client.threadPool().schedule(wrappedListener, timeoutInterval, ThreadPool.Names.GENERIC);
             listenerToReturn = wrappedListener;
@@ -100,6 +106,22 @@ public static <Response> ActionListener<Response> wrapWithCancellationListener(
         return listenerToReturn;
     }
 
+    /**
+     * A functional interface used to handle the timeout of a cancellable task.
+     * Implementations of this interface provide a callback method that is invoked
+     * when a task is cancelled due to a timeout.
+     */
+    public interface TimeoutHandler {
+
+        /**
+         * Called when a cancellable task is cancelled due to a timeout.
+         *
+         * @param e the exception that contains details about the task cancellation,
+         *          including the reason for cancellation.
+         */
+        void onTimeout(TaskCancelledException e);
+    }
+
     /**
      * Timeout listener which executes the provided runnable after timeout is expired and if a response/failure is not yet received.
      * If either a response/failure is received before timeout then the scheduled task is cancelled and response/failure is sent back to
@@ -144,6 +166,7 @@ public void run() {
                 if (executeRunnable.compareAndSet(true, false)) {
                     timeoutRunnable.run();
                 } // else do nothing since either response/failure is already sent to client
+
             } catch (Exception ex) {
                 // ignore the exception
                 logger.error(
diff --git a/server/src/main/java/org/opensearch/action/support/broadcast/BroadcastRequest.java b/server/src/main/java/org/opensearch/action/support/broadcast/BroadcastRequest.java
index 8a27e032cec5e..784c43c938f2c 100644
--- a/server/src/main/java/org/opensearch/action/support/broadcast/BroadcastRequest.java
+++ b/server/src/main/java/org/opensearch/action/support/broadcast/BroadcastRequest.java
@@ -52,6 +52,8 @@ public class BroadcastRequest<Request extends BroadcastRequest<Request>> extends
     protected String[] indices;
     private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpenAndForbidClosed();
 
+    protected boolean shouldCancelOnTimeout = false;
+
     public BroadcastRequest(StreamInput in) throws IOException {
         super(in);
         indices = in.readStringArray();
@@ -125,4 +127,12 @@ public void writeTo(StreamOutput out) throws IOException {
         out.writeStringArrayNullable(indices);
         indicesOptions.writeIndicesOptions(out);
     }
+
+    public void setShouldCancelOnTimeout(boolean shouldCancelOnTimeout) {
+        this.shouldCancelOnTimeout = shouldCancelOnTimeout;
+    }
+
+    public boolean getShouldCancelOnTimeout() {
+        return this.shouldCancelOnTimeout;
+    }
 }
diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeReadRequest.java b/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeReadRequest.java
index 6dcc6ed1b098e..c373107163082 100644
--- a/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeReadRequest.java
+++ b/server/src/main/java/org/opensearch/action/support/clustermanager/ClusterManagerNodeReadRequest.java
@@ -47,6 +47,8 @@ public abstract class ClusterManagerNodeReadRequest<Request extends ClusterManag
 
     protected boolean local = false;
 
+    protected boolean shouldCancelOnTimeout = false;
+
     protected ClusterManagerNodeReadRequest() {}
 
     protected ClusterManagerNodeReadRequest(StreamInput in) throws IOException {
@@ -74,4 +76,12 @@ public final Request local(boolean local) {
     public final boolean local() {
         return local;
     }
+
+    public void setShouldCancelOnTimeout(boolean shouldCancelOnTimeout) {
+        this.shouldCancelOnTimeout = shouldCancelOnTimeout;
+    }
+
+    public boolean getShouldCancelOnTimeout() {
+        return this.shouldCancelOnTimeout;
+    }
 }
diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/ClusterAdminTask.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/ClusterAdminTask.java
new file mode 100644
index 0000000000000..5361b05ebecc3
--- /dev/null
+++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/ClusterAdminTask.java
@@ -0,0 +1,47 @@
+/*
+ * 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.rest.action.admin.cluster;
+
+import org.opensearch.common.annotation.PublicApi;
+import org.opensearch.common.unit.TimeValue;
+import org.opensearch.core.tasks.TaskId;
+import org.opensearch.tasks.CancellableTask;
+
+import java.util.Map;
+
+import static org.opensearch.search.SearchService.NO_TIMEOUT;
+
+/**
+ * Task storing information about a currently running ClusterRequest.
+ *
+ * @opensearch.api
+ */
+@PublicApi(since = "2.17.0")
+public class ClusterAdminTask extends CancellableTask {
+
+    public ClusterAdminTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
+        this(id, type, action, parentTaskId, headers, NO_TIMEOUT);
+    }
+
+    public ClusterAdminTask(
+        long id,
+        String type,
+        String action,
+        TaskId parentTaskId,
+        Map<String, String> headers,
+        TimeValue cancelAfterTimeInterval
+    ) {
+        super(id, type, action, null, parentTaskId, headers, cancelAfterTimeInterval);
+    }
+
+    @Override
+    public boolean shouldCancelChildrenOnCancellation() {
+        return true;
+    }
+}
diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java
index dedf6dc305cb6..1bdd6f4921b7a 100644
--- a/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java
+++ b/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java
@@ -32,10 +32,11 @@
 
 package org.opensearch.rest.action.cat;
 
-import org.opensearch.action.admin.cluster.state.ClusterStateRequest;
+import org.opensearch.action.admin.cluster.shards.CatShardsAction;
+import org.opensearch.action.admin.cluster.shards.CatShardsRequest;
+import org.opensearch.action.admin.cluster.shards.CatShardsResponse;
 import org.opensearch.action.admin.cluster.state.ClusterStateResponse;
 import org.opensearch.action.admin.indices.stats.CommonStats;
-import org.opensearch.action.admin.indices.stats.IndicesStatsRequest;
 import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
 import org.opensearch.action.admin.indices.stats.ShardStats;
 import org.opensearch.client.node.NodeClient;
@@ -61,7 +62,6 @@
 import org.opensearch.index.warmer.WarmerStats;
 import org.opensearch.rest.RestRequest;
 import org.opensearch.rest.RestResponse;
-import org.opensearch.rest.action.RestActionListener;
 import org.opensearch.rest.action.RestResponseListener;
 import org.opensearch.search.suggest.completion.CompletionStats;
 
@@ -73,6 +73,7 @@
 import static java.util.Arrays.asList;
 import static java.util.Collections.unmodifiableList;
 import static org.opensearch.rest.RestRequest.Method.GET;
+import static org.opensearch.search.SearchService.NO_TIMEOUT;
 
 /**
  * _cat API action to get shard information
@@ -107,25 +108,18 @@ protected void documentation(StringBuilder sb) {
     @Override
     public RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) {
         final String[] indices = Strings.splitStringByCommaToArray(request.param("index"));
-        final ClusterStateRequest clusterStateRequest = new ClusterStateRequest();
-        clusterStateRequest.local(request.paramAsBoolean("local", clusterStateRequest.local()));
-        clusterStateRequest.clusterManagerNodeTimeout(
-            request.paramAsTime("cluster_manager_timeout", clusterStateRequest.clusterManagerNodeTimeout())
-        );
-        parseDeprecatedMasterTimeoutParameter(clusterStateRequest, request);
-        clusterStateRequest.clear().nodes(true).routingTable(true).indices(indices);
-        return channel -> client.admin().cluster().state(clusterStateRequest, new RestActionListener<ClusterStateResponse>(channel) {
+        final CatShardsRequest shardsRequest = new CatShardsRequest();
+        shardsRequest.local(request.paramAsBoolean("local", shardsRequest.local()));
+        shardsRequest.clusterManagerNodeTimeout(request.paramAsTime("cluster_manager_timeout", shardsRequest.clusterManagerNodeTimeout()));
+        shardsRequest.setCancelAfterTimeInterval(request.paramAsTime("cancel_after_time_interval", NO_TIMEOUT));
+        shardsRequest.setIndices(indices);
+        parseDeprecatedMasterTimeoutParameter(shardsRequest, request);
+        return channel -> client.execute(CatShardsAction.INSTANCE, shardsRequest, new RestResponseListener<CatShardsResponse>(channel) {
             @Override
-            public void processResponse(final ClusterStateResponse clusterStateResponse) {
-                IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest();
-                indicesStatsRequest.all();
-                indicesStatsRequest.indices(indices);
-                client.admin().indices().stats(indicesStatsRequest, new RestResponseListener<IndicesStatsResponse>(channel) {
-                    @Override
-                    public RestResponse buildResponse(IndicesStatsResponse indicesStatsResponse) throws Exception {
-                        return RestTable.buildResponse(buildTable(request, clusterStateResponse, indicesStatsResponse), channel);
-                    }
-                });
+            public RestResponse buildResponse(CatShardsResponse catShardsResponse) throws Exception {
+                ClusterStateResponse clusterStateResponse = catShardsResponse.getClusterStateResponse();
+                IndicesStatsResponse indicesStatsResponse = catShardsResponse.getIndicesStatsResponse();
+                return RestTable.buildResponse(buildTable(request, clusterStateResponse, indicesStatsResponse), channel);
             }
         });
     }
diff --git a/server/src/test/java/org/opensearch/action/admin/indices/shards/CatShardsResponseTests.java b/server/src/test/java/org/opensearch/action/admin/indices/shards/CatShardsResponseTests.java
new file mode 100644
index 0000000000000..d0b98f4d286ae
--- /dev/null
+++ b/server/src/test/java/org/opensearch/action/admin/indices/shards/CatShardsResponseTests.java
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Modifications Copyright OpenSearch Contributors. See
+ * GitHub history for details.
+ */
+
+package org.opensearch.action.admin.indices.shards;
+
+import org.opensearch.action.admin.cluster.shards.CatShardsResponse;
+import org.opensearch.action.admin.cluster.state.ClusterStateResponse;
+import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
+import org.opensearch.cluster.ClusterName;
+import org.opensearch.test.OpenSearchTestCase;
+
+import static org.junit.Assert.assertEquals;
+
+public class CatShardsResponseTests extends OpenSearchTestCase {
+
+    private final CatShardsResponse catShardsResponse = new CatShardsResponse();
+
+    public void testGetAndSetClusterStateResponse() {
+        ClusterName clusterName = new ClusterName("1");
+        ClusterStateResponse clusterStateResponse = new ClusterStateResponse(clusterName, null, false);
+        catShardsResponse.setClusterStateResponse(clusterStateResponse);
+
+        assertEquals(clusterStateResponse, catShardsResponse.getClusterStateResponse());
+    }
+
+    public void testGetAndSetIndicesStatsResponse() {
+        final IndicesStatsResponse indicesStatsResponse = new IndicesStatsResponse(null, 0, 0, 0, null);
+        catShardsResponse.setIndicesStatsResponse(indicesStatsResponse);
+
+        assertEquals(indicesStatsResponse, catShardsResponse.getIndicesStatsResponse());
+    }
+}