Skip to content

Commit 38b3bc4

Browse files
authored
Limit concurrent shard requests in disk usage API (#84900) (#85366)
The disk_usage API fails on indices that have more than 16 shards. The reason is that we broadcast all shard requests simultaneously, and the handling thread pool - the analyzer thread pool - whose queue size is only 16. That means some shard requests will be rejected. This PR limits the number of concurrent shard requests sent to each node when we broadcast them. Closes #84779
1 parent 12c32fa commit 38b3bc4

File tree

5 files changed

+577
-19
lines changed

5 files changed

+577
-19
lines changed

docs/changelog/84900.yaml

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,6 @@
1+
pr: 84900
2+
summary: Limit concurrent shard requests in disk usage API
3+
area: Search
4+
type: bug
5+
issues:
6+
- 84779

server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/diskusage/IndexDiskUsageAnalyzerIT.java

Lines changed: 47 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010

1111
import org.apache.lucene.geo.GeoTestUtil;
1212
import org.apache.lucene.util.English;
13+
import org.elasticsearch.action.support.IndicesOptions;
1314
import org.elasticsearch.action.support.PlainActionFuture;
1415
import org.elasticsearch.cluster.metadata.IndexMetadata;
1516
import org.elasticsearch.common.settings.Settings;
@@ -28,13 +29,15 @@
2829
import org.elasticsearch.xcontent.XContentFactory;
2930
import org.junit.Before;
3031

32+
import java.util.Arrays;
3133
import java.util.Collection;
3234
import java.util.List;
3335
import java.util.Optional;
3436
import java.util.Set;
3537
import java.util.stream.Collectors;
3638
import java.util.stream.IntStream;
3739

40+
import static org.hamcrest.Matchers.emptyArray;
3841
import static org.hamcrest.Matchers.equalTo;
3942
import static org.hamcrest.Matchers.greaterThan;
4043

@@ -240,6 +243,50 @@ public void testFailOnFlush() throws Exception {
240243
assertThat(resp.getFailedShards(), equalTo(failedShards.size()));
241244
}
242245

246+
public void testManyShards() throws Exception {
247+
List<String> indices = IntStream.range(0, between(1, 5)).mapToObj(n -> "index_" + n).collect(Collectors.toList());
248+
int totalShards = 0;
249+
for (String indexName : indices) {
250+
int numberOfShards = between(10, 30);
251+
client().admin()
252+
.indices()
253+
.prepareCreate(indexName)
254+
.setSettings(
255+
Settings.builder()
256+
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numberOfShards)
257+
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, between(0, 1))
258+
.put("index.shard.check_on_startup", false)
259+
.put("index.routing.rebalance.enable", "none")
260+
)
261+
.get();
262+
totalShards += numberOfShards;
263+
int numDocs = randomIntBetween(10, 100);
264+
for (int i = 0; i < numDocs; i++) {
265+
int value = randomIntBetween(5, 20);
266+
final XContentBuilder doc = XContentFactory.jsonBuilder()
267+
.startObject()
268+
.field("english_text", English.intToEnglish(value))
269+
.field("value", value)
270+
.endObject();
271+
client().prepareIndex(indexName, "_doc").setId("id-" + i).setSource(doc).get();
272+
}
273+
}
274+
275+
AnalyzeIndexDiskUsageResponse resp = client().execute(
276+
AnalyzeIndexDiskUsageAction.INSTANCE,
277+
new AnalyzeIndexDiskUsageRequest(new String[] { "index_*" }, IndicesOptions.fromOptions(false, false, true, true), true)
278+
).actionGet();
279+
assertThat(Arrays.toString(resp.getShardFailures()), resp.getShardFailures(), emptyArray());
280+
assertThat(resp.getTotalShards(), equalTo(totalShards));
281+
assertThat(resp.getSuccessfulShards(), equalTo(totalShards));
282+
assertThat(resp.getFailedShards(), equalTo(0));
283+
for (String index : indices) {
284+
IndexDiskUsageStats stats = resp.getStats().get(index);
285+
assertThat(stats.getIndexSizeInBytes(), greaterThan(0L));
286+
assertThat(stats.total().totalBytes(), greaterThan(0L));
287+
}
288+
}
289+
243290
void assertMetadataFields(IndexDiskUsageStats stats) {
244291
final IndexDiskUsageStats.PerFieldDiskUsage sourceField = stats.getFields().get("_source");
245292
assertThat(sourceField.getInvertedIndexBytes(), equalTo(0L));

server/src/main/java/org/elasticsearch/action/admin/indices/diskusage/TransportAnalyzeIndexDiskUsageAction.java

Lines changed: 93 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,14 @@
1818
import org.elasticsearch.cluster.block.ClusterBlockException;
1919
import org.elasticsearch.cluster.block.ClusterBlockLevel;
2020
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
21+
import org.elasticsearch.cluster.node.DiscoveryNode;
2122
import org.elasticsearch.cluster.routing.GroupShardsIterator;
2223
import org.elasticsearch.cluster.routing.ShardIterator;
2324
import org.elasticsearch.cluster.routing.ShardRouting;
2425
import org.elasticsearch.cluster.service.ClusterService;
2526
import org.elasticsearch.common.inject.Inject;
2627
import org.elasticsearch.common.io.stream.StreamInput;
28+
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
2729
import org.elasticsearch.index.engine.Engine;
2830
import org.elasticsearch.index.shard.IndexShard;
2931
import org.elasticsearch.index.shard.ShardId;
@@ -36,8 +38,12 @@
3638
import java.io.IOException;
3739
import java.util.ArrayList;
3840
import java.util.HashMap;
41+
import java.util.Iterator;
42+
import java.util.LinkedList;
3943
import java.util.List;
4044
import java.util.Map;
45+
import java.util.Queue;
46+
import java.util.concurrent.atomic.AtomicInteger;
4147
import java.util.concurrent.atomic.AtomicReferenceArray;
4248

4349
public class TransportAnalyzeIndexDiskUsageAction extends TransportBroadcastAction<
@@ -46,6 +52,7 @@ public class TransportAnalyzeIndexDiskUsageAction extends TransportBroadcastActi
4652
AnalyzeDiskUsageShardRequest,
4753
AnalyzeDiskUsageShardResponse> {
4854
private final IndicesService indicesService;
55+
private final ThreadPool threadPool;
4956

5057
@Inject
5158
public TransportAnalyzeIndexDiskUsageAction(
@@ -66,11 +73,96 @@ public TransportAnalyzeIndexDiskUsageAction(
6673
ThreadPool.Names.ANALYZE
6774
);
6875
this.indicesService = indexServices;
76+
this.threadPool = transportService.getThreadPool();
6977
}
7078

7179
@Override
7280
protected void doExecute(Task task, AnalyzeIndexDiskUsageRequest request, ActionListener<AnalyzeIndexDiskUsageResponse> listener) {
73-
super.doExecute(task, request, listener);
81+
new LimitingRequestPerNodeBroadcastAction(task, request, listener, 5).start();
82+
}
83+
84+
private static class ShardRequest {
85+
private final DiscoveryNode node;
86+
private final AnalyzeDiskUsageShardRequest shardRequest;
87+
private final ActionListener<AnalyzeDiskUsageShardResponse> handler;
88+
89+
ShardRequest(DiscoveryNode node, AnalyzeDiskUsageShardRequest shardRequest, ActionListener<AnalyzeDiskUsageShardResponse> handler) {
90+
this.node = node;
91+
this.shardRequest = shardRequest;
92+
this.handler = handler;
93+
}
94+
}
95+
96+
final class LimitingRequestPerNodeBroadcastAction extends AsyncBroadcastAction {
97+
private final Queue<ShardRequest> queue = new LinkedList<>();
98+
private final Map<DiscoveryNode, AtomicInteger> sendingCounters = ConcurrentCollections.newConcurrentMap();
99+
private final int maxConcurrentRequestsPerNode;
100+
101+
LimitingRequestPerNodeBroadcastAction(
102+
Task task,
103+
AnalyzeIndexDiskUsageRequest request,
104+
ActionListener<AnalyzeIndexDiskUsageResponse> listener,
105+
int maxConcurrentRequestsPerNode
106+
) {
107+
super(task, request, listener);
108+
this.maxConcurrentRequestsPerNode = maxConcurrentRequestsPerNode;
109+
}
110+
111+
private void trySendRequests() {
112+
assert Thread.holdsLock(this) == false;
113+
final List<ShardRequest> readyRequests = new ArrayList<>();
114+
synchronized (this) {
115+
final Iterator<ShardRequest> it = queue.iterator();
116+
while (it.hasNext()) {
117+
final ShardRequest r = it.next();
118+
final AtomicInteger sending = sendingCounters.computeIfAbsent(r.node, k -> new AtomicInteger());
119+
assert 0 <= sending.get() && sending.get() <= maxConcurrentRequestsPerNode : sending;
120+
if (sending.get() < maxConcurrentRequestsPerNode) {
121+
sending.incrementAndGet();
122+
readyRequests.add(r);
123+
it.remove();
124+
}
125+
}
126+
}
127+
if (readyRequests.isEmpty()) {
128+
return;
129+
}
130+
final Thread sendingThread = Thread.currentThread();
131+
for (ShardRequest r : readyRequests) {
132+
super.sendShardRequest(
133+
r.node,
134+
r.shardRequest,
135+
ActionListener.runAfter(r.handler, () -> onRequestResponded(sendingThread, r.node))
136+
);
137+
}
138+
}
139+
140+
private void onRequestResponded(Thread sendingThread, DiscoveryNode node) {
141+
final AtomicInteger sending = sendingCounters.get(node);
142+
assert sending != null && 1 <= sending.get() && sending.get() <= maxConcurrentRequestsPerNode : sending;
143+
sending.decrementAndGet();
144+
// fork to avoid StackOverflow
145+
if (sendingThread == Thread.currentThread()) {
146+
threadPool.generic().execute(this::trySendRequests);
147+
} else {
148+
trySendRequests();
149+
}
150+
}
151+
152+
@Override
153+
protected synchronized void sendShardRequest(
154+
DiscoveryNode node,
155+
AnalyzeDiskUsageShardRequest shardRequest,
156+
ActionListener<AnalyzeDiskUsageShardResponse> listener
157+
) {
158+
queue.add(new ShardRequest(node, shardRequest, listener));
159+
}
160+
161+
@Override
162+
public void start() {
163+
super.start();
164+
trySendRequests();
165+
}
74166
}
75167

76168
@Override

server/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java

Lines changed: 21 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -171,26 +171,10 @@ protected void performOperation(final ShardIterator shardIt, final ShardRouting
171171
// no node connected, act as failure
172172
onOperation(shard, shardIt, shardIndex, new NoShardAvailableActionException(shardIt.shardId()));
173173
} else {
174-
transportService.sendRequest(
174+
sendShardRequest(
175175
node,
176-
transportShardAction,
177176
shardRequest,
178-
new TransportResponseHandler<ShardResponse>() {
179-
@Override
180-
public ShardResponse read(StreamInput in) throws IOException {
181-
return readShardResponse(in);
182-
}
183-
184-
@Override
185-
public void handleResponse(ShardResponse response) {
186-
onOperation(shard, shardIndex, response);
187-
}
188-
189-
@Override
190-
public void handleException(TransportException e) {
191-
onOperation(shard, shardIt, shardIndex, e);
192-
}
193-
}
177+
ActionListener.wrap(r -> onOperation(shard, shardIndex, r), e -> onOperation(shard, shardIt, shardIndex, e))
194178
);
195179
}
196180
} catch (Exception e) {
@@ -199,6 +183,25 @@ public void handleException(TransportException e) {
199183
}
200184
}
201185

186+
protected void sendShardRequest(DiscoveryNode node, ShardRequest shardRequest, ActionListener<ShardResponse> listener) {
187+
transportService.sendRequest(node, transportShardAction, shardRequest, new TransportResponseHandler<ShardResponse>() {
188+
@Override
189+
public ShardResponse read(StreamInput in) throws IOException {
190+
return readShardResponse(in);
191+
}
192+
193+
@Override
194+
public void handleResponse(ShardResponse response) {
195+
listener.onResponse(response);
196+
}
197+
198+
@Override
199+
public void handleException(TransportException e) {
200+
listener.onFailure(e);
201+
}
202+
});
203+
}
204+
202205
protected void onOperation(ShardRouting shard, int shardIndex, ShardResponse response) {
203206
logger.trace("received response for {}", shard);
204207
shardsResponses.set(shardIndex, response);

0 commit comments

Comments
 (0)