diff --git a/rest-api-spec/build.gradle b/rest-api-spec/build.gradle index 029f54e2116c3..a208936776b6c 100644 --- a/rest-api-spec/build.gradle +++ b/rest-api-spec/build.gradle @@ -108,4 +108,5 @@ tasks.named("yamlRestCompatTestTransform").configure ({ task -> "cluster.desired_nodes/10_basic/Test update desired nodes with node_version generates a warning", "node_version warning is removed in 9.0" ) + task.skipTest("cluster.info/30_info_thread_pool/Cluster HTTP Info", "The search_throttled thread pool has been removed") }) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.info/30_info_thread_pool.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.info/30_info_thread_pool.yml index 02a5666639543..67c3484b81f00 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.info/30_info_thread_pool.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.info/30_info_thread_pool.yml @@ -95,13 +95,6 @@ - gte: { thread_pool.search_coordination.largest: 0 } - gte: { thread_pool.search_coordination.completed: 0 } - - gte: { thread_pool.search_throttled.threads: 0 } - - gte: { thread_pool.search_throttled.queue: 0 } - - gte: { thread_pool.search_throttled.active: 0 } - - gte: { thread_pool.search_throttled.rejected: 0 } - - gte: { thread_pool.search_throttled.largest: 0 } - - gte: { thread_pool.search_throttled.completed: 0 } - - gte: { thread_pool.snapshot.threads: 0 } - gte: { thread_pool.snapshot.queue: 0 } - gte: { thread_pool.snapshot.active: 0 } diff --git a/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java b/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java index 84c6df7b8a66f..e7274ea7ca80a 100644 --- a/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java +++ b/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java @@ -44,7 +44,6 @@ import java.io.IOException; import java.util.Set; -import java.util.concurrent.Executor; import java.util.function.LongSupplier; /** @@ -182,12 +181,4 @@ protected ShardIterator shards(ClusterState state, InternalRequest request) { request.request().preference() ); } - - @Override - protected Executor getExecutor(ExplainRequest request, ShardId shardId) { - IndexService indexService = searchService.getIndicesService().indexServiceSafe(shardId.getIndex()); - return indexService.getIndexSettings().isSearchThrottled() - ? threadPool.executor(ThreadPool.Names.SEARCH_THROTTLED) - : super.getExecutor(request, shardId); - } } diff --git a/server/src/main/java/org/elasticsearch/action/get/TransportGetAction.java b/server/src/main/java/org/elasticsearch/action/get/TransportGetAction.java index 29b926598ac32..ede2fdb910948 100644 --- a/server/src/main/java/org/elasticsearch/action/get/TransportGetAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/TransportGetAction.java @@ -166,8 +166,6 @@ protected Executor getExecutor(GetRequest request, ShardId shardId) { final ClusterState clusterState = clusterService.state(); if (clusterState.metadata().getIndexSafe(shardId.getIndex()).isSystem()) { return threadPool.executor(executorSelector.executorForGet(shardId.getIndexName())); - } else if (indicesService.indexServiceSafe(shardId.getIndex()).getIndexSettings().isSearchThrottled()) { - return threadPool.executor(ThreadPool.Names.SEARCH_THROTTLED); } else { return super.getExecutor(request, shardId); } diff --git a/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java b/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java index d9a04acc0466e..a407f7be5c1fa 100644 --- a/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetAction.java @@ -155,8 +155,6 @@ protected Executor getExecutor(MultiGetShardRequest request, ShardId shardId) { final ClusterState clusterState = clusterService.state(); if (clusterState.metadata().index(shardId.getIndex()).isSystem()) { return threadPool.executor(executorSelector.executorForGet(shardId.getIndexName())); - } else if (indicesService.indexServiceSafe(shardId.getIndex()).getIndexSettings().isSearchThrottled()) { - return threadPool.executor(ThreadPool.Names.SEARCH_THROTTLED); } else { return super.getExecutor(request, shardId); } diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java b/server/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java index 7840af1baad3a..6a51728375aba 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TransportShardMultiTermsVectorAction.java @@ -27,8 +27,6 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import java.util.concurrent.Executor; - import static org.elasticsearch.core.Strings.format; public class TransportShardMultiTermsVectorAction extends TransportSingleShardAction< @@ -109,12 +107,4 @@ protected MultiTermVectorsShardResponse shardOperation(MultiTermVectorsShardRequ return response; } - - @Override - protected Executor getExecutor(MultiTermVectorsShardRequest request, ShardId shardId) { - IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex()); - return indexService.getIndexSettings().isSearchThrottled() - ? threadPool.executor(ThreadPool.Names.SEARCH_THROTTLED) - : super.getExecutor(request, shardId); - } } diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java b/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java index 02479a9f8d143..6f22c950093af 100644 --- a/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java +++ b/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java @@ -28,7 +28,6 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.concurrent.Executor; /** * Performs the get operation. @@ -125,12 +124,4 @@ protected TermVectorsResponse shardOperation(TermVectorsRequest request, ShardId protected Writeable.Reader getResponseReader() { return TermVectorsResponse::new; } - - @Override - protected Executor getExecutor(TermVectorsRequest request, ShardId shardId) { - IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex()); - return indexService.getIndexSettings().isSearchThrottled() - ? threadPool.executor(ThreadPool.Names.SEARCH_THROTTLED) - : super.getExecutor(request, shardId); - } } diff --git a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index 35289352d7daf..adb4b584fb8e3 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -150,7 +150,6 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING, IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING, IndexSettings.INDEX_SEARCH_IDLE_AFTER, - IndexSettings.INDEX_SEARCH_THROTTLED, IndexFieldDataService.INDEX_FIELDDATA_CACHE_KEY, IndexSettings.IGNORE_ABOVE_SETTING, FieldMapper.IGNORE_MALFORMED_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java index 4895930eaefe4..66deea71148f2 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -536,17 +536,6 @@ public Iterator> settings() { Property.ServerlessPublic ); - /** - * Marks an index to be searched throttled. This means that never more than one shard of such an index will be searched concurrently - */ - public static final Setting INDEX_SEARCH_THROTTLED = Setting.boolSetting( - "index.search.throttled", - false, - Property.IndexScope, - Property.PrivateIndex, - Property.Dynamic - ); - /** * Determines a balance between file-based and operations-based peer recoveries. The number of operations that will be used in an * operations-based peer recovery is limited to this proportion of the total number of documents in the shard (including deleted @@ -891,7 +880,6 @@ private void setRetentionLeaseMillis(final TimeValue retentionLease) { private volatile int maxTermsCount; private volatile String defaultPipeline; private volatile String requiredPipeline; - private volatile boolean searchThrottled; private volatile long mappingNestedFieldsLimit; private volatile long mappingNestedDocsLimit; private volatile long mappingTotalFieldsLimit; @@ -1019,7 +1007,6 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti this.timestampBounds = TimestampBounds.updateEndTime(this.timestampBounds, endTime); }); } - this.searchThrottled = INDEX_SEARCH_THROTTLED.get(settings); this.queryStringLenient = QUERY_STRING_LENIENT_SETTING.get(settings); this.queryStringAnalyzeWildcard = QUERY_STRING_ANALYZE_WILDCARD.get(nodeSettings); this.queryStringAllowLeadingWildcard = QUERY_STRING_ALLOW_LEADING_WILDCARD.get(nodeSettings); @@ -1175,7 +1162,6 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti scopedSettings.addSettingsUpdateConsumer(DEFAULT_PIPELINE, this::setDefaultPipeline); scopedSettings.addSettingsUpdateConsumer(FINAL_PIPELINE, this::setRequiredPipeline); scopedSettings.addSettingsUpdateConsumer(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, this::setSoftDeleteRetentionOperations); - scopedSettings.addSettingsUpdateConsumer(INDEX_SEARCH_THROTTLED, this::setSearchThrottled); scopedSettings.addSettingsUpdateConsumer(INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING, this::setRetentionLeaseMillis); scopedSettings.addSettingsUpdateConsumer(INDEX_MAPPING_NESTED_FIELDS_LIMIT_SETTING, this::setMappingNestedFieldsLimit); scopedSettings.addSettingsUpdateConsumer(INDEX_MAPPING_NESTED_DOCS_LIMIT_SETTING, this::setMappingNestedDocsLimit); @@ -1700,18 +1686,6 @@ public long getSoftDeleteRetentionOperations() { return this.softDeleteRetentionOperations; } - /** - * Returns true if the this index should be searched throttled ie. using the - * {@link org.elasticsearch.threadpool.ThreadPool.Names#SEARCH_THROTTLED} thread-pool - */ - public boolean isSearchThrottled() { - return searchThrottled; - } - - private void setSearchThrottled(boolean searchThrottled) { - this.searchThrottled = searchThrottled; - } - public long getMappingNestedFieldsLimit() { return mappingNestedFieldsLimit; } diff --git a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java index 9ba21a7d77342..fb167bf5a61ca 100644 --- a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java +++ b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java @@ -566,6 +566,14 @@ private static void addBwcSearchWorkerSettings(List> additionalSettin additionalSettings.add( Setting.intSetting("thread_pool.search_worker.size", 0, Setting.Property.NodeScope, Setting.Property.DeprecatedWarning) ); + // Search throttled thread pool has been removed in Elasticsearch 9.0.0. These settings are deprecated and take no effect. + // They are here only to enable BwC for deployments that still use them + additionalSettings.add( + Setting.intSetting("thread_pool.search_throttled.queue_size", 0, Setting.Property.NodeScope, Setting.Property.DeprecatedWarning) + ); + additionalSettings.add( + Setting.intSetting("thread_pool.search_throttled.size", 0, Setting.Property.NodeScope, Setting.Property.DeprecatedWarning) + ); } private SearchModule createSearchModule(Settings settings, ThreadPool threadPool, TelemetryProvider telemetryProvider) { diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java index 703f1f2c18408..f5027216c38cd 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.core.TimeValue; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.Scope; @@ -117,7 +116,6 @@ public RestResponse buildResponse(Void ignored) throws Exception { .prepareGetSettings(indices) .setIndicesOptions(indicesOptions) .setMasterNodeTimeout(masterNodeTimeout) - .setNames(IndexSettings.INDEX_SEARCH_THROTTLED.getKey()) .execute(listeners.acquire(indexSettingsRef::set)); // The other requests just provide additional detail, and wildcards may be resolved differently depending on the type of @@ -480,8 +478,6 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("memory.total", "sibling:pri;alias:tm,memoryTotal;default:false;text-align:right;desc:total used memory"); table.addCell("pri.memory.total", "default:false;text-align:right;desc:total user memory"); - table.addCell("search.throttled", "alias:sth;default:false;desc:indicates if the index is search throttled"); - table.addCell( "bulk.total_operations", "sibling:pri;alias:bto,bulkTotalOperation;default:false;text-align:right;desc:number of bulk shard ops" @@ -553,7 +549,6 @@ Table buildTable( final IndexMetadata.State indexState = indexMetadata.getState(); final IndexStats indexStats = indicesStats.get(indexName); - final boolean searchThrottled = IndexSettings.INDEX_SEARCH_THROTTLED.get(settings); final IndexRoutingTable indexRoutingTable = clusterState.routingTable().index(indexName); final ClusterHealthStatus indexHealthStatus = indexRoutingTable == null @@ -788,8 +783,6 @@ Table buildTable( table.addCell(totalStats.getTotalMemory()); table.addCell(primaryStats.getTotalMemory()); - table.addCell(searchThrottled); - table.addCell(totalStats.getBulk() == null ? null : totalStats.getBulk().getTotalOperations()); table.addCell(primaryStats.getBulk() == null ? null : primaryStats.getBulk().getTotalOperations()); diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index efa27b2f3448c..1dfd6ed6bb683 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -908,8 +908,6 @@ private Executor getExecutor(IndexShard indexShard) { final String executorName; if (indexShard.isSystem()) { executorName = executorSelector.executorForSearch(indexShard.shardId().getIndexName()); - } else if (indexShard.indexSettings().isSearchThrottled()) { - executorName = Names.SEARCH_THROTTLED; } else { executorName = Names.SEARCH; } diff --git a/server/src/main/java/org/elasticsearch/threadpool/DefaultBuiltInExecutorBuilders.java b/server/src/main/java/org/elasticsearch/threadpool/DefaultBuiltInExecutorBuilders.java index 32634043cfc98..9698ce6b65cdf 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/DefaultBuiltInExecutorBuilders.java +++ b/server/src/main/java/org/elasticsearch/threadpool/DefaultBuiltInExecutorBuilders.java @@ -92,10 +92,6 @@ public Map getBuilders(Settings settings, int allocated EsExecutors.TaskTrackingConfig.DEFAULT ) ); - result.put( - ThreadPool.Names.SEARCH_THROTTLED, - new FixedExecutorBuilder(settings, ThreadPool.Names.SEARCH_THROTTLED, 1, 100, EsExecutors.TaskTrackingConfig.DEFAULT) - ); result.put( ThreadPool.Names.MANAGEMENT, new ScalingExecutorBuilder( diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index 52f0ec6a39d65..96d82793a3f4f 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -115,8 +115,6 @@ public static class Names { public static final String SEARCH = "search"; public static final String SEARCH_COORDINATION = "search_coordination"; public static final String AUTO_COMPLETE = "auto_complete"; - public static final String SEARCH_THROTTLED = "search_throttled"; - /** * A thread pool for running tasks related to cluster management, including collecting and exposing stats in APIs and certain other * internal tasks. @@ -197,7 +195,6 @@ public static ThreadPoolType fromType(String type) { entry(Names.FORCE_MERGE, ThreadPoolType.FIXED), entry(Names.FETCH_SHARD_STARTED, ThreadPoolType.SCALING), entry(Names.FETCH_SHARD_STORE, ThreadPoolType.SCALING), - entry(Names.SEARCH_THROTTLED, ThreadPoolType.FIXED), entry(Names.SYSTEM_READ, ThreadPoolType.FIXED), entry(Names.SYSTEM_WRITE, ThreadPoolType.FIXED), entry(Names.SYSTEM_CRITICAL_READ, ThreadPoolType.FIXED), diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java index 293bdb2c53899..92b72fc92f042 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -30,7 +30,6 @@ import org.elasticsearch.core.Tuple; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.indices.IndexClosedException; import org.elasticsearch.indices.InvalidIndexNameException; @@ -2269,7 +2268,7 @@ public void testIgnoreThrottled() { .putAlias(AliasMetadata.builder("test-alias")) ) .put( - indexBuilder("index", Settings.builder().put(IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), true).build()).state(State.OPEN) + indexBuilder("index", Settings.builder().put("index.search.throttled", true).build()).state(State.OPEN) .putAlias(AliasMetadata.builder("test-alias2")) ) .put( diff --git a/server/src/test/java/org/elasticsearch/rest/action/cat/RestIndicesActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/cat/RestIndicesActionTests.java index 5187a31492457..993c61c50c8b6 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/cat/RestIndicesActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/cat/RestIndicesActionTests.java @@ -25,7 +25,6 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; @@ -58,7 +57,6 @@ public void testBuildTable() { Settings indexSettings = Settings.builder() .put(IndexMetadata.SETTING_VERSION_CREATED, IndexVersion.current()) .put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) - .put(IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), randomBoolean()) .build(); indicesSettings.put(indexName, indexSettings); diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java index fe602d2854c8c..01095f9869cee 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java @@ -58,7 +58,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.AbstractRefCounted; import org.elasticsearch.core.TimeValue; -import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexService; @@ -167,7 +166,6 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; @@ -1846,46 +1844,6 @@ public void testCanRewriteToMatchNone() { ); } - public void testSetSearchThrottled() throws IOException { - createIndex("throttled_threadpool_index"); - client().execute( - InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, - new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request( - "throttled_threadpool_index", - IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), - "true" - ) - ).actionGet(); - final SearchService service = getInstanceFromNode(SearchService.class); - Index index = resolveIndex("throttled_threadpool_index"); - assertTrue(service.getIndicesService().indexServiceSafe(index).getIndexSettings().isSearchThrottled()); - prepareIndex("throttled_threadpool_index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - assertSearchHits( - client().prepareSearch("throttled_threadpool_index") - .setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED) - .setSize(1), - "1" - ); - // we add a search action listener in a plugin above to assert that this is actually used - client().execute( - InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, - new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request( - "throttled_threadpool_index", - IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), - "false" - ) - ).actionGet(); - - IllegalArgumentException iae = expectThrows( - IllegalArgumentException.class, - () -> indicesAdmin().prepareUpdateSettings("throttled_threadpool_index") - .setSettings(Settings.builder().put(IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), false)) - .get() - ); - assertEquals("can not update private setting [index.search.throttled]; this setting is managed by Elasticsearch", iae.getMessage()); - assertFalse(service.getIndicesService().indexServiceSafe(index).getIndexSettings().isSearchThrottled()); - } - public void testAggContextGetsMatchAll() throws IOException { createIndex("test"); withAggregationContext("test", context -> assertThat(context.query(), equalTo(new MatchAllDocsQuery()))); @@ -1937,22 +1895,6 @@ private void withAggregationContext(String index, Consumer c } } - public void testExpandSearchThrottled() { - createIndex("throttled_threadpool_index"); - client().execute( - InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, - new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request( - "throttled_threadpool_index", - IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), - "true" - ) - ).actionGet(); - - prepareIndex("throttled_threadpool_index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - assertHitCount(client().prepareSearch(), 1L); - assertHitCount(client().prepareSearch().setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED), 1L); - } - public void testExpandSearchFrozen() { String indexName = "frozen_index"; createIndex(indexName); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java index f6323d3cd46c8..5b138ae07625c 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java @@ -485,7 +485,6 @@ static String[] extractLeaderShardHistoryUUIDs(Map ccrIndexMetad IndexSettings.MAX_SLICES_PER_SCROLL, IndexSettings.DEFAULT_PIPELINE, IndexSettings.FINAL_PIPELINE, - IndexSettings.INDEX_SEARCH_THROTTLED, IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING, IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING, IndexSettings.INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING, diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/input/MetadataCachingIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/input/MetadataCachingIndexInput.java index c1468f2e45df0..0c2942f800d42 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/input/MetadataCachingIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/input/MetadataCachingIndexInput.java @@ -221,7 +221,6 @@ public static boolean assertCurrentThreadMayAccessBlobStore() { ThreadPool.Names.SNAPSHOT, ThreadPool.Names.GENERIC, ThreadPool.Names.SEARCH, - ThreadPool.Names.SEARCH_THROTTLED, // Cache asynchronous fetching runs on a dedicated thread pool. SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME,