From 1c473017d339c3a95e51ce4e87bc2750ed5a6102 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 7 Jun 2018 15:46:18 +0200 Subject: [PATCH 1/3] Default max concurrent search req. to the avg shards size per index We moved to 1 shard by default which caused some issues in how many concurrent shard requests we allow by default. For instance searching a 5 shard index on a single node will now be executed serially per shard while we want these cases to have a good concurrency out of the box. This change moves to defaults based on the avg. number of shards per index in the current search request to provide a good out of the box concurrency. Relates to #30783 Closes #30994 --- .../elasticsearch/action/search/TransportSearchAction.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 46207b94c3af4..51ee8b045bfee 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -345,8 +345,13 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea * We try to set a default of max concurrent shard requests based on the node count but upper-bound it by 256 by default to keep * it sane. A single search request that fans out to lots of shards should not hit a cluster too hard while 256 is already a * lot. + * Yet we are still trying to guarantee some concurrency ie. if you have an index size of N shards and you are searching a + * single index on a single node we still search it concurrently. */ - searchRequest.setMaxConcurrentShardRequests(Math.min(256, nodeCount)); + final int numShards = shardIterators.size(); + final int numIndices = remoteClusterIndices.size() + concreteIndices.length; + assert numShards >= numIndices : "found schroedingers index numShards: " + numShards + " vs. numIndices: " + numIndices; + searchRequest.setMaxConcurrentShardRequests(Math.min(256, nodeCount * (numShards / numIndices))); } boolean preFilterSearchShards = shouldPreFilterSearchShards(searchRequest, shardIterators); searchAsyncAction(task, searchRequest, shardIterators, timeProvider, connectionLookup, clusterState.version(), From aafae7a9d5fa143538100dc025a5942755e76f4d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 7 Jun 2018 16:19:38 +0200 Subject: [PATCH 2/3] fix devision by zero --- .../org/elasticsearch/action/search/TransportSearchAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 51ee8b045bfee..2687172d6cde7 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -340,7 +340,7 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea } return searchTransportService.getConnection(clusterName, discoveryNode); }; - if (searchRequest.isMaxConcurrentShardRequestsSet() == false) { + if (searchRequest.isMaxConcurrentShardRequestsSet() == false && shardIterators.size() > 0) { /* * We try to set a default of max concurrent shard requests based on the node count but upper-bound it by 256 by default to keep * it sane. A single search request that fans out to lots of shards should not hit a cluster too hard while 256 is already a @@ -351,6 +351,7 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea final int numShards = shardIterators.size(); final int numIndices = remoteClusterIndices.size() + concreteIndices.length; assert numShards >= numIndices : "found schroedingers index numShards: " + numShards + " vs. numIndices: " + numIndices; + assert numIndices > 0; searchRequest.setMaxConcurrentShardRequests(Math.min(256, nodeCount * (numShards / numIndices))); } boolean preFilterSearchShards = shouldPreFilterSearchShards(searchRequest, shardIterators); From 62ee2d10691e2d6478bfcef0976487b5de04e1ef Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 7 Jun 2018 21:36:47 +0200 Subject: [PATCH 3/3] move to a constant --- .../action/search/TransportSearchAction.java | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 2687172d6cde7..ac9248ef98d41 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -340,19 +340,14 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea } return searchTransportService.getConnection(clusterName, discoveryNode); }; - if (searchRequest.isMaxConcurrentShardRequestsSet() == false && shardIterators.size() > 0) { + if (searchRequest.isMaxConcurrentShardRequestsSet() == false) { /* * We try to set a default of max concurrent shard requests based on the node count but upper-bound it by 256 by default to keep * it sane. A single search request that fans out to lots of shards should not hit a cluster too hard while 256 is already a * lot. - * Yet we are still trying to guarantee some concurrency ie. if you have an index size of N shards and you are searching a - * single index on a single node we still search it concurrently. */ - final int numShards = shardIterators.size(); - final int numIndices = remoteClusterIndices.size() + concreteIndices.length; - assert numShards >= numIndices : "found schroedingers index numShards: " + numShards + " vs. numIndices: " + numIndices; - assert numIndices > 0; - searchRequest.setMaxConcurrentShardRequests(Math.min(256, nodeCount * (numShards / numIndices))); + // we use nodeCount * 5 as we used to default this to the default number of shard which used to be 5. + searchRequest.setMaxConcurrentShardRequests(Math.min(256, nodeCount * 5)); } boolean preFilterSearchShards = shouldPreFilterSearchShards(searchRequest, shardIterators); searchAsyncAction(task, searchRequest, shardIterators, timeProvider, connectionLookup, clusterState.version(),