From a1ffb915b9610c8ac1778d7411c73924f40d2504 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 29 Jul 2020 17:36:44 -0600 Subject: [PATCH 001/161] first cut of header-based system index enforcement --- .../ReindexSourceTargetValidationTests.java | 4 +- .../action/IndicesRequestIT.java | 4 +- .../DedicatedClusterSnapshotRestoreIT.java | 3 +- .../elasticsearch/cluster/ClusterModule.java | 5 +- .../cluster/metadata/IndexAbstraction.java | 20 ++ .../metadata/IndexNameExpressionResolver.java | 76 ++++-- .../java/org/elasticsearch/node/Node.java | 2 +- .../elasticsearch/rest/BaseRestHandler.java | 5 + .../action/ActionModuleTests.java | 22 +- ...tAddVotingConfigExclusionsActionTests.java | 3 +- ...learVotingConfigExclusionsActionTests.java | 3 +- .../indices/get/GetIndexActionTests.java | 5 + .../mapping/put/PutMappingRequestTests.java | 17 +- .../indices/resolve/ResolveIndexTests.java | 4 +- .../MetadataRolloverServiceTests.java | 3 +- .../settings/get/GetSettingsActionTests.java | 5 + .../bulk/TransportBulkActionIngestTests.java | 3 +- .../bulk/TransportBulkActionTookTests.java | 5 + .../get/TransportMultiGetActionTests.java | 5 + .../search/MultiSearchActionTookTests.java | 5 + .../action/support/AutoCreateIndexTests.java | 6 +- .../TransportBroadcastByNodeActionTests.java | 5 + .../TransportMasterNodeActionTests.java | 4 +- .../BroadcastReplicationTests.java | 3 +- ...ortInstanceSingleOperationActionTests.java | 6 + .../TransportMultiTermVectorsActionTests.java | 5 + .../cluster/ClusterModuleTests.java | 17 +- .../health/ClusterStateHealthTests.java | 4 +- .../DateMathExpressionResolverTests.java | 6 +- ...ExpressionResolverAliasIterationTests.java | 7 +- ...sionResolverExpressionsIterationTests.java | 7 +- .../IndexNameExpressionResolverTests.java | 228 +++++++++++++----- .../WildcardExpressionResolverTests.java | 34 +-- .../elasticsearch/index/IndexModuleTests.java | 9 +- .../query/SearchIndexNameMatcherTests.java | 7 +- .../indices/cluster/ClusterStateChanges.java | 3 +- .../rest/BaseRestHandlerTests.java | 38 ++- .../snapshots/SnapshotResiliencyTests.java | 4 +- .../test/rest/ESRestTestCase.java | 1 + .../core/ilm/GenerateSnapshotNameStep.java | 2 +- .../xpack/core/ml/utils/MlIndexAndAlias.java | 2 +- .../validation/SourceDestValidatorTests.java | 17 +- .../DeprecationInfoActionResponseTests.java | 3 +- .../core/ml/utils/MlIndexAndAliasTests.java | 3 +- .../GetDataStreamsTransportActionTests.java | 54 ++++- .../xpack/enrich/AbstractEnrichTestCase.java | 4 +- .../enrich/EnrichPolicyExecutorTests.java | 5 +- .../EnrichPolicyMaintenanceServiceTests.java | 3 +- .../AutodetectResultProcessorIT.java | 4 +- .../ml/integration/EstablishedMemUsageIT.java | 3 +- .../ml/integration/JobResultsProviderIT.java | 3 +- .../integration/JobStorageDeletionTaskIT.java | 3 +- .../ml/integration/MlAutoUpdateServiceIT.java | 3 +- .../ml/integration/MlConfigMigratorIT.java | 3 +- .../ml/integration/UnusedStatsRemoverIT.java | 3 +- .../action/TransportOpenJobActionTests.java | 11 +- ...ortStartDataFrameAnalyticsActionTests.java | 6 +- .../DatafeedConfigAutoUpdaterTests.java | 3 +- .../datafeed/DatafeedNodeSelectorTests.java | 4 +- .../TrainedModelStatsServiceTests.java | 3 +- .../persistence/JobResultsProviderTests.java | 2 +- .../AutodetectProcessManagerTests.java | 2 +- .../cluster/ClusterStatsCollectorTests.java | 5 +- .../DateMathExpressionIntegTests.java | 5 +- .../xpack/security/SecurityTests.java | 2 +- .../authz/AuthorizationServiceTests.java | 10 +- .../authz/IndicesAndAliasesResolverTests.java | 6 +- ...TransformPersistentTasksExecutorTests.java | 9 +- .../xpack/watcher/WatcherPluginTests.java | 3 +- 69 files changed, 560 insertions(+), 219 deletions(-) diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexSourceTargetValidationTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexSourceTargetValidationTests.java index df08634f72287..c4315437af321 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexSourceTargetValidationTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexSourceTargetValidationTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.test.ESTestCase; import static java.util.Collections.emptyMap; @@ -58,7 +59,8 @@ public class ReindexSourceTargetValidationTests extends ESTestCase { .put(index("baz"), true) .put(index("source", "source_multi"), true) .put(index("source2", "source_multi"), true)).build(); - private static final IndexNameExpressionResolver INDEX_NAME_EXPRESSION_RESOLVER = new IndexNameExpressionResolver(); + private static final IndexNameExpressionResolver INDEX_NAME_EXPRESSION_RESOLVER = + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); private static final AutoCreateIndex AUTO_CREATE_INDEX = new AutoCreateIndex(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), INDEX_NAME_EXPRESSION_RESOLVER); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java index e11f97d9c3498..e95072dbb4a15 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java @@ -392,7 +392,7 @@ public void testFlush() { internalCluster().coordOnlyNodeClient().admin().indices().flush(flushRequest).actionGet(); clearInterceptedActions(); - String[] indices = new IndexNameExpressionResolver() + String[] indices = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)) .concreteIndexNames(client().admin().cluster().prepareState().get().getState(), flushRequest); assertIndicesSubset(Arrays.asList(indices), indexShardActions); } @@ -417,7 +417,7 @@ public void testRefresh() { internalCluster().coordOnlyNodeClient().admin().indices().refresh(refreshRequest).actionGet(); clearInterceptedActions(); - String[] indices = new IndexNameExpressionResolver() + String[] indices = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)) .concreteIndexNames(client().admin().cluster().prepareState().get().getState(), refreshRequest); assertIndicesSubset(Arrays.asList(indices), indexShardActions); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index 0555434a03584..dbe944fd1b05f 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -60,6 +60,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; @@ -886,7 +887,7 @@ public void testSnapshotWithDateMath() { final String repo = "repo"; final AdminClient admin = client().admin(); - final IndexNameExpressionResolver nameExpressionResolver = new IndexNameExpressionResolver(); + final IndexNameExpressionResolver nameExpressionResolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); final String snapshotName = ""; logger.info("--> creating repository"); diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java index addca81c329c6..4d2aeef2919a8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java @@ -77,6 +77,7 @@ import org.elasticsearch.script.ScriptMetadata; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskResultsService; +import org.elasticsearch.threadpool.ThreadPool; import java.util.ArrayList; import java.util.Collection; @@ -107,13 +108,13 @@ public class ClusterModule extends AbstractModule { final ShardsAllocator shardsAllocator; public ClusterModule(Settings settings, ClusterService clusterService, List clusterPlugins, - ClusterInfoService clusterInfoService) { + ClusterInfoService clusterInfoService, ThreadPool threadPool) { this.clusterPlugins = clusterPlugins; this.deciderList = createAllocationDeciders(settings, clusterService.getClusterSettings(), clusterPlugins); this.allocationDeciders = new AllocationDeciders(deciderList); this.shardsAllocator = createShardsAllocator(settings, clusterService.getClusterSettings(), clusterPlugins); this.clusterService = clusterService; - this.indexNameExpressionResolver = new IndexNameExpressionResolver(); + this.indexNameExpressionResolver = new IndexNameExpressionResolver(threadPool.getThreadContext()); this.allocationService = new AllocationService(allocationDeciders, shardsAllocator, clusterInfoService); } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java index ec2c6c1c33b81..833de1fa979d9 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java @@ -78,6 +78,8 @@ public interface IndexAbstraction { */ boolean isHidden(); + boolean isSystem(); + /** * An index abstraction type. */ @@ -160,6 +162,11 @@ public DataStream getParentDataStream() { public boolean isHidden() { return INDEX_HIDDEN_SETTING.get(concreteIndex.getSettings()); } + + @Override + public boolean isSystem() { + return concreteIndex.isSystem(); + } } /** @@ -210,6 +217,13 @@ public boolean isHidden() { return isHidden; } + @Override + public boolean isSystem() { + //G-> This is probably not the best way to tell if an alias is a system-index-alias + // this should probably be checked on the alias layer and added as a property in the AliasMetadata + return referenceIndexMetadatas.stream().anyMatch(IndexMetadata::isSystem); // G-> + } + /** * Returns the unique alias metadata per concrete index. *

@@ -325,6 +339,12 @@ public boolean isHidden() { return false; } + @Override + public boolean isSystem() { + // No such thing as system data streams (yet) + return false; + } + public org.elasticsearch.cluster.metadata.DataStream getDataStream() { return dataStream; } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 67b891002a653..bacc4e001988d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.time.DateMathParser; import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; @@ -66,12 +67,19 @@ public class IndexNameExpressionResolver { private final WildcardExpressionResolver wildcardExpressionResolver = new WildcardExpressionResolver(); private final List expressionResolvers = List.of(dateMathExpressionResolver, wildcardExpressionResolver); + private final ThreadContext threadContext; + + public IndexNameExpressionResolver(ThreadContext threadContext) { + this.threadContext = Objects.requireNonNull(threadContext, "Thread Context must not be null"); + } + /** * Same as {@link #concreteIndexNames(ClusterState, IndicesOptions, String...)}, but the index expressions and options * are encapsulated in the specified request. */ public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { - Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams()); + Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), + isSystemIndexAccessAllowed()); return concreteIndexNames(context, request.indices()); } @@ -80,7 +88,8 @@ public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { * are encapsulated in the specified request and resolves data streams. */ public Index[] concreteIndices(ClusterState state, IndicesRequest request) { - Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams()); + Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), + isSystemIndexAccessAllowed()); return concreteIndices(context, request.indices()); } @@ -98,22 +107,22 @@ public Index[] concreteIndices(ClusterState state, IndicesRequest request) { * indices options in the context don't allow such a case. */ public String[] concreteIndexNames(ClusterState state, IndicesOptions options, String... indexExpressions) { - Context context = new Context(state, options); + Context context = new Context(state, options, isSystemIndexAccessAllowed()); return concreteIndexNames(context, indexExpressions); } public String[] concreteIndexNames(ClusterState state, IndicesOptions options, boolean includeDataStreams, String... indexExpressions) { - Context context = new Context(state, options, false, false, includeDataStreams); + Context context = new Context(state, options, false, false, includeDataStreams, isSystemIndexAccessAllowed()); return concreteIndexNames(context, indexExpressions); } public String[] concreteIndexNames(ClusterState state, IndicesOptions options, IndicesRequest request) { - Context context = new Context(state, options, false, false, request.includeDataStreams()); + Context context = new Context(state, options, false, false, request.includeDataStreams(), isSystemIndexAccessAllowed()); return concreteIndexNames(context, request.indices()); } public List dataStreamNames(ClusterState state, IndicesOptions options, String... indexExpressions) { - Context context = new Context(state, options, false, false, true, true); + Context context = new Context(state, options, false, false, true, true, isSystemIndexAccessAllowed()); if (indexExpressions == null || indexExpressions.length == 0) { indexExpressions = new String[]{"*"}; } @@ -145,7 +154,7 @@ public Index[] concreteIndices(ClusterState state, IndicesOptions options, Strin } public Index[] concreteIndices(ClusterState state, IndicesOptions options, boolean includeDataStreams, String... indexExpressions) { - Context context = new Context(state, options, false, false, includeDataStreams); + Context context = new Context(state, options, false, false, includeDataStreams, isSystemIndexAccessAllowed()); return concreteIndices(context, indexExpressions); } @@ -162,7 +171,8 @@ public Index[] concreteIndices(ClusterState state, IndicesOptions options, boole * indices options in the context don't allow such a case. */ public Index[] concreteIndices(ClusterState state, IndicesRequest request, long startTime) { - Context context = new Context(state, request.indicesOptions(), startTime, false, false, request.includeDataStreams(), false); + Context context = new Context(state, request.indicesOptions(), startTime, false, false, request.includeDataStreams(), false, + isSystemIndexAccessAllowed()); return concreteIndices(context, request.indices()); } @@ -286,6 +296,10 @@ Index[] concreteIndices(Context context, String... indexExpressions) { } private static boolean shouldTrackConcreteIndex(Context context, IndicesOptions options, IndexMetadata index) { + if (context.isSystemIndexAccessAllowed() == false && index.isSystem()) { + return false; + } + if (index.getState() == IndexMetadata.State.CLOSE) { if (options.forbidClosedIndices() && options.ignoreUnavailable() == false) { throw new IndexClosedException(index.getIndex()); @@ -369,7 +383,7 @@ public Index concreteWriteIndex(ClusterState state, IndicesOptions options, Stri options.allowAliasesToMultipleIndices(), options.forbidClosedIndices(), options.ignoreAliases(), options.ignoreThrottled()); - Context context = new Context(state, combinedOptions, false, true, includeDataStreams); + Context context = new Context(state, combinedOptions, false, true, includeDataStreams, isSystemIndexAccessAllowed()); Index[] indices = concreteIndices(context, index); if (allowNoIndices && indices.length == 0) { return null; @@ -386,7 +400,7 @@ public Index concreteWriteIndex(ClusterState state, IndicesOptions options, Stri * If the data stream, index or alias contains date math then that is resolved too. */ public boolean hasIndexAbstraction(String indexAbstraction, ClusterState state) { - Context context = new Context(state, IndicesOptions.lenientExpandOpen(), false, false, true); + Context context = new Context(state, IndicesOptions.lenientExpandOpen(), false, false, true, isSystemIndexAccessAllowed()); String resolvedAliasOrIndex = dateMathExpressionResolver.resolveExpression(indexAbstraction, context); return state.metadata().getIndicesLookup().containsKey(resolvedAliasOrIndex); } @@ -397,14 +411,14 @@ public boolean hasIndexAbstraction(String indexAbstraction, ClusterState state) public String resolveDateMathExpression(String dateExpression) { // The data math expression resolver doesn't rely on cluster state or indices options, because // it just resolves the date math to an actual date. - return dateMathExpressionResolver.resolveExpression(dateExpression, new Context(null, null)); + return dateMathExpressionResolver.resolveExpression(dateExpression, new Context(null, null, isSystemIndexAccessAllowed())); } /** * Resolve an array of expressions to the set of indices and aliases that these expressions match. */ public Set resolveExpressions(ClusterState state, String... expressions) { - Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true, false, true); + Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true, false, true, isSystemIndexAccessAllowed()); List resolvedExpressions = Arrays.asList(expressions); for (ExpressionResolver expressionResolver : expressionResolvers) { resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions); @@ -498,7 +512,7 @@ public String[] indexAliases(ClusterState state, String index, Predicate> resolveSearchRouting(ClusterState state, @Nullable String routing, String... expressions) { List resolvedExpressions = expressions != null ? Arrays.asList(expressions) : Collections.emptyList(); - Context context = new Context(state, IndicesOptions.lenientExpandOpen(), false, false, true); + Context context = new Context(state, IndicesOptions.lenientExpandOpen(), false, false, true, isSystemIndexAccessAllowed()); for (ExpressionResolver expressionResolver : expressionResolvers) { resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions); } @@ -650,6 +664,13 @@ boolean isPatternMatchingAllIndices(Metadata metadata, String[] indicesOrAliases return false; } + private boolean isSystemIndexAccessAllowed() { + if (threadContext == null) { + return false; + } + return "true".equals(threadContext.getHeader("_from_rest")) == false; + } + public static class Context { private final ClusterState state; @@ -659,27 +680,30 @@ public static class Context { private final boolean resolveToWriteIndex; private final boolean includeDataStreams; private final boolean preserveDataStreams; + private final boolean isSystemIndexAccessAllowed; - Context(ClusterState state, IndicesOptions options) { - this(state, options, System.currentTimeMillis()); + Context(ClusterState state, IndicesOptions options, boolean isSystemIndexAccessAllowed) { + this(state, options, System.currentTimeMillis(), isSystemIndexAccessAllowed); } Context(ClusterState state, IndicesOptions options, boolean preserveAliases, boolean resolveToWriteIndex, - boolean includeDataStreams) { - this(state, options, System.currentTimeMillis(), preserveAliases, resolveToWriteIndex, includeDataStreams, false); + boolean includeDataStreams, boolean isSystemIndexAccessAllowed) { + this(state, options, System.currentTimeMillis(), preserveAliases, resolveToWriteIndex, includeDataStreams, false, + isSystemIndexAccessAllowed); } Context(ClusterState state, IndicesOptions options, boolean preserveAliases, boolean resolveToWriteIndex, - boolean includeDataStreams, boolean preserveDataStreams) { - this(state, options, System.currentTimeMillis(), preserveAliases, resolveToWriteIndex, includeDataStreams, preserveDataStreams); + boolean includeDataStreams, boolean preserveDataStreams, boolean isSystemIndexAccessAllowed) { + this(state, options, System.currentTimeMillis(), preserveAliases, resolveToWriteIndex, includeDataStreams, preserveDataStreams, + isSystemIndexAccessAllowed); } - Context(ClusterState state, IndicesOptions options, long startTime) { - this(state, options, startTime, false, false, false, false); + Context(ClusterState state, IndicesOptions options, long startTime, boolean isSystemIndexAccessAllowed) { + this(state, options, startTime, false, false, false, false, isSystemIndexAccessAllowed); } protected Context(ClusterState state, IndicesOptions options, long startTime, boolean preserveAliases, boolean resolveToWriteIndex, - boolean includeDataStreams, boolean preserveDataStreams) { + boolean includeDataStreams, boolean preserveDataStreams, boolean isSystemIndexAccessAllowed) { this.state = state; this.options = options; this.startTime = startTime; @@ -687,6 +711,7 @@ protected Context(ClusterState state, IndicesOptions options, long startTime, bo this.resolveToWriteIndex = resolveToWriteIndex; this.includeDataStreams = includeDataStreams; this.preserveDataStreams = preserveDataStreams; + this.isSystemIndexAccessAllowed = isSystemIndexAccessAllowed; } public ClusterState getState() { @@ -725,6 +750,10 @@ public boolean includeDataStreams() { public boolean isPreserveDataStreams() { return preserveDataStreams; } + + public boolean isSystemIndexAccessAllowed() { + return isSystemIndexAccessAllowed; + } } private interface ExpressionResolver { @@ -954,6 +983,9 @@ private static Set expand(Context context, IndexMetadata.State excludeSt for (Map.Entry entry : matches.entrySet()) { String aliasOrIndexName = entry.getKey(); IndexAbstraction indexAbstraction = entry.getValue(); + if (context.isSystemIndexAccessAllowed() == false && indexAbstraction.isSystem()) { + continue; + } if (indexAbstraction.isHidden() == false || includeHidden || implicitHiddenMatch(aliasOrIndexName, expression)) { if (context.isPreserveAliases() && indexAbstraction.getType() == IndexAbstraction.Type.ALIAS) { diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 8be8504698998..e15b8c264df4b 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -395,7 +395,7 @@ protected Node(final Environment initialEnvironment, final MonitorService monitorService = new MonitorService(settings, nodeEnvironment, threadPool); final FsHealthService fsHealthService = new FsHealthService(settings, clusterService.getClusterSettings(), threadPool, nodeEnvironment); - ClusterModule clusterModule = new ClusterModule(settings, clusterService, clusterPlugins, clusterInfoService); + ClusterModule clusterModule = new ClusterModule(settings, clusterService, clusterPlugins, clusterInfoService, threadPool); modules.add(clusterModule); IndicesModule indicesModule = new IndicesModule(pluginsService.filterPlugins(MapperPlugin.class)); modules.add(indicesModule); diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 101c3182fbb62..fdf474bcccb4b 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -76,6 +76,11 @@ public final long getUsageCount() { @Override public final void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { + final String allow_system_index_access = request.param("allow_system_index_access"); + if (allow_system_index_access != null + && (allow_system_index_access.isEmpty() || Boolean.parseBoolean(allow_system_index_access) == false)) { + client.threadPool().getThreadContext().putHeader("_from_rest", "true"); + } // prepare the request for execution; has the side effect of touching the request parameters final RestChannelConsumer action = prepareRequest(request, client); diff --git a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java index 544b0b6f38ef9..c8a19842f6e6b 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.settings.SettingsModule; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.ActionPlugin.ActionHandler; import org.elasticsearch.rest.RestChannel; @@ -107,9 +108,10 @@ protected FakeAction() { public void testSetupRestHandlerContainsKnownBuiltin() { SettingsModule settings = new SettingsModule(Settings.EMPTY); UsageService usageService = new UsageService(); - ActionModule actionModule = new ActionModule(settings.getSettings(), new IndexNameExpressionResolver(), - settings.getIndexScopedSettings(), settings.getClusterSettings(), settings.getSettingsFilter(), null, emptyList(), null, - null, usageService); + ActionModule actionModule = new ActionModule(settings.getSettings(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), settings.getIndexScopedSettings(), + settings.getClusterSettings(), settings.getSettingsFilter(), null, emptyList(), null, + null, usageService); actionModule.initRestHandlers(null); // At this point the easiest way to confirm that a handler is loaded is to try to register another one on top of it and to fail Exception e = expectThrows(IllegalArgumentException.class, () -> @@ -146,9 +148,10 @@ public String getName() { ThreadPool threadPool = new TestThreadPool(getTestName()); try { UsageService usageService = new UsageService(); - ActionModule actionModule = new ActionModule(settings.getSettings(), new IndexNameExpressionResolver(), - settings.getIndexScopedSettings(), settings.getClusterSettings(), settings.getSettingsFilter(), threadPool, - singletonList(dupsMainAction), null, null, usageService); + ActionModule actionModule = new ActionModule(settings.getSettings(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), settings.getIndexScopedSettings(), + settings.getClusterSettings(), settings.getSettingsFilter(), threadPool, singletonList(dupsMainAction), + null, null, usageService); Exception e = expectThrows(IllegalArgumentException.class, () -> actionModule.initRestHandlers(null)); assertThat(e.getMessage(), startsWith("Cannot replace existing handler for [/] for method: GET")); } finally { @@ -180,9 +183,10 @@ public List getRestHandlers(Settings settings, RestController restC ThreadPool threadPool = new TestThreadPool(getTestName()); try { UsageService usageService = new UsageService(); - ActionModule actionModule = new ActionModule(settings.getSettings(), new IndexNameExpressionResolver(), - settings.getIndexScopedSettings(), settings.getClusterSettings(), settings.getSettingsFilter(), threadPool, - singletonList(registersFakeHandler), null, null, usageService); + ActionModule actionModule = new ActionModule(settings.getSettings(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), settings.getIndexScopedSettings(), + settings.getClusterSettings(), settings.getSettingsFilter(), threadPool, singletonList(registersFakeHandler), + null, null, usageService); actionModule.initRestHandlers(null); // At this point the easiest way to confirm that a handler is loaded is to try to register another one on top of it and to fail Exception e = expectThrows(IllegalArgumentException.class, () -> diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java index 221a9a1114c65..1343c30749b20 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java @@ -41,6 +41,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransport; import org.elasticsearch.threadpool.TestThreadPool; @@ -131,7 +132,7 @@ public void setupForTest() { clusterSettings = new ClusterSettings(nodeSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); new TransportAddVotingConfigExclusionsAction(nodeSettings, clusterSettings, transportService, clusterService, threadPool, - new ActionFilters(emptySet()), new IndexNameExpressionResolver()); // registers action + new ActionFilters(emptySet()), new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); // registers action transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsActionTests.java index 4c7fcbe7e2f56..e7fd68b6c0d0a 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/configuration/TransportClearVotingConfigExclusionsActionTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransport; import org.elasticsearch.threadpool.TestThreadPool; @@ -95,7 +96,7 @@ public void setupForTest() { TransportService.NOOP_TRANSPORT_INTERCEPTOR, boundTransportAddress -> localNode, null, emptySet()); new TransportClearVotingConfigExclusionsAction(transportService, clusterService, threadPool, new ActionFilters(emptySet()), - new IndexNameExpressionResolver()); // registers action + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); // registers action transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java index 0ff854af20c54..1594fa1bff5e1 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexActionTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.settings.SettingsModule; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -122,6 +123,10 @@ protected void doMasterOperation(GetIndexRequest request, String[] concreteIndic } static class Resolver extends IndexNameExpressionResolver { + Resolver() { + super(new ThreadContext(Settings.EMPTY)); + } + @Override public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { return request.indices(); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestTests.java index 9b50d2ee2f93b..b3d54c200a78c 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequestTests.java @@ -28,6 +28,8 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; import org.elasticsearch.test.ESTestCase; @@ -89,7 +91,8 @@ public void testResolveIndicesWithWriteIndexOnlyAndDataStreamsAndWriteAliases() tuple("alias2", List.of(tuple("index2", false), tuple("index3", true))) )); PutMappingRequest request = new PutMappingRequest().indices("foo", "alias1", "alias2").writeIndexOnly(true); - Index[] indices = TransportPutMappingAction.resolveIndices(cs, request, new IndexNameExpressionResolver()); + Index[] indices = TransportPutMappingAction.resolveIndices(cs, request, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); List indexNames = Arrays.stream(indices).map(Index::getName).collect(Collectors.toList()); IndexAbstraction expectedDs = cs.metadata().getIndicesLookup().get("foo"); // should resolve the data stream and each alias to their respective write indices @@ -109,7 +112,8 @@ public void testResolveIndicesWithoutWriteIndexOnlyAndDataStreamsAndWriteAliases tuple("alias2", List.of(tuple("index2", false), tuple("index3", true))) )); PutMappingRequest request = new PutMappingRequest().indices("foo", "alias1", "alias2"); - Index[] indices = TransportPutMappingAction.resolveIndices(cs, request, new IndexNameExpressionResolver()); + Index[] indices = TransportPutMappingAction.resolveIndices(cs, request, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); List indexNames = Arrays.stream(indices).map(Index::getName).collect(Collectors.toList()); IndexAbstraction expectedDs = cs.metadata().getIndicesLookup().get("foo"); List expectedIndices = expectedDs.getIndices().stream().map(im -> im.getIndex().getName()).collect(Collectors.toList()); @@ -131,7 +135,8 @@ public void testResolveIndicesWithWriteIndexOnlyAndDataStreamAndIndex() { tuple("alias2", List.of(tuple("index2", false), tuple("index3", true))) )); PutMappingRequest request = new PutMappingRequest().indices("foo", "index3").writeIndexOnly(true); - Index[] indices = TransportPutMappingAction.resolveIndices(cs, request, new IndexNameExpressionResolver()); + Index[] indices = TransportPutMappingAction.resolveIndices(cs, request, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); List indexNames = Arrays.stream(indices).map(Index::getName).collect(Collectors.toList()); IndexAbstraction expectedDs = cs.metadata().getIndicesLookup().get("foo"); List expectedIndices = expectedDs.getIndices().stream().map(im -> im.getIndex().getName()).collect(Collectors.toList()); @@ -154,7 +159,8 @@ public void testResolveIndicesWithWriteIndexOnlyAndNoSingleWriteIndex() { )); PutMappingRequest request = new PutMappingRequest().indices("*").writeIndexOnly(true); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> TransportPutMappingAction.resolveIndices(cs2, request, new IndexNameExpressionResolver())); + () -> TransportPutMappingAction.resolveIndices(cs2, request, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)))); assertThat(e.getMessage(), containsString("The index expression [*] and options provided did not point to a single write-index")); } @@ -172,7 +178,8 @@ public void testResolveIndicesWithWriteIndexOnlyAndAliasWithoutWriteIndex() { )); PutMappingRequest request = new PutMappingRequest().indices("alias2").writeIndexOnly(true); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> TransportPutMappingAction.resolveIndices(cs2, request, new IndexNameExpressionResolver())); + () -> TransportPutMappingAction.resolveIndices(cs2, request, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)))); assertThat(e.getMessage(), containsString("no write index is defined for alias [alias2]")); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/resolve/ResolveIndexTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/resolve/ResolveIndexTests.java index 5a33643202d09..acf6d97f5878e 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/resolve/ResolveIndexTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/resolve/ResolveIndexTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.test.ESTestCase; import java.util.ArrayList; @@ -69,7 +70,8 @@ public class ResolveIndexTests extends ESTestCase { }; private Metadata metadata = buildMetadata(dataStreams, indices); - private IndexAbstractionResolver resolver = new IndexAbstractionResolver(new IndexNameExpressionResolver()); + private IndexAbstractionResolver resolver = new IndexAbstractionResolver( + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); public void testResolveStarWithDefaultOptions() { String[] names = new String[] {"*"}; diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java index a12d9d85789e0..0db37c2eb6365 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/MetadataRolloverServiceTests.java @@ -51,6 +51,7 @@ import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.env.Environment; import org.elasticsearch.index.Index; @@ -296,7 +297,7 @@ public void testDataStreamValidation() throws IOException { public void testGenerateRolloverIndexName() { String invalidIndexName = randomAlphaOfLength(10) + "A"; - IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); expectThrows(IllegalArgumentException.class, () -> MetadataRolloverService.generateRolloverIndexName(invalidIndexName, indexNameExpressionResolver)); int num = randomIntBetween(0, 100); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsActionTests.java index db443cf97f126..8975cbac98656 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/settings/get/GetSettingsActionTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.settings.SettingsModule; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESTestCase; @@ -132,6 +133,10 @@ public void testIncludeDefaultsWithFiltering() { } static class Resolver extends IndexNameExpressionResolver { + Resolver() { + super(new ThreadContext(Settings.EMPTY)); + } + @Override public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { return request.indices(); diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java index 8e7af9de16adb..acc9ffeb23734 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionIngestTests.java @@ -50,6 +50,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexingPressure; @@ -143,7 +144,7 @@ class TestTransportBulkAction extends TransportBulkAction { null, new ActionFilters(Collections.emptySet()), null, new AutoCreateIndex( SETTINGS, new ClusterSettings(SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), - new IndexNameExpressionResolver() + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)) ), new IndexingPressure(SETTINGS) ); } diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java index 44007488433db..38a0b8cebacaa 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexingPressure; @@ -208,6 +209,10 @@ public void onFailure(Exception e) { } static class Resolver extends IndexNameExpressionResolver { + Resolver() { + super(new ThreadContext(Settings.EMPTY)); + } + @Override public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { return request.indices(); diff --git a/server/src/test/java/org/elasticsearch/action/get/TransportMultiGetActionTests.java b/server/src/test/java/org/elasticsearch/action/get/TransportMultiGetActionTests.java index 3a0f3ba116c9a..32e7de7f402b9 100644 --- a/server/src/test/java/org/elasticsearch/action/get/TransportMultiGetActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/get/TransportMultiGetActionTests.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; @@ -223,6 +224,10 @@ private static Task createTask() { static class Resolver extends IndexNameExpressionResolver { + Resolver() { + super(new ThreadContext(Settings.EMPTY)); + } + @Override public Index concreteSingleIndex(ClusterState state, IndicesRequest request) { return new Index("index1", randomBase64UUID()); diff --git a/server/src/test/java/org/elasticsearch/action/search/MultiSearchActionTookTests.java b/server/src/test/java/org/elasticsearch/action/search/MultiSearchActionTookTests.java index 19b53e2f8d380..fab9fd33c05ac 100644 --- a/server/src/test/java/org/elasticsearch/action/search/MultiSearchActionTookTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/MultiSearchActionTookTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskManager; @@ -191,6 +192,10 @@ void executeSearch(final Queue requests, final AtomicArray getResults() { } class MyResolver extends IndexNameExpressionResolver { + MyResolver() { + super(new ThreadContext(Settings.EMPTY)); + } + @Override public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { return request.indices(); diff --git a/server/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java b/server/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java index 77f617ca3f9fd..05bad21c4f0f6 100644 --- a/server/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java @@ -43,7 +43,9 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.discovery.MasterNotDiscoveredException; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.rest.RestStatus; @@ -172,7 +174,7 @@ class Action extends TransportMasterNodeAction { Action(String actionName, TransportService transportService, ClusterService clusterService, ThreadPool threadPool) { super(actionName, transportService, clusterService, threadPool, - new ActionFilters(new HashSet<>()), Request::new, new IndexNameExpressionResolver()); + new ActionFilters(new HashSet<>()), Request::new, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java index f2f376de3ca72..e154ea86f84a0 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.breaker.CircuitBreakerService; @@ -103,7 +104,7 @@ threadPool, new NetworkService(Collections.emptyList()), PageCacheRecycler.NON_R transportService.start(); transportService.acceptIncomingRequests(); broadcastReplicationAction = new TestBroadcastReplicationAction(clusterService, transportService, - new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver()); + new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java index ad8e9d3943a0b..987d51de9d98e 100644 --- a/server/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java @@ -39,7 +39,9 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; @@ -136,6 +138,10 @@ protected ShardIterator shards(ClusterState clusterState, Request request) { } class MyResolver extends IndexNameExpressionResolver { + MyResolver() { + super(new ThreadContext(Settings.EMPTY)); + } + @Override public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { return request.indices(); diff --git a/server/src/test/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsActionTests.java b/server/src/test/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsActionTests.java index ba51419bfb928..92fe3f16f0718 100644 --- a/server/src/test/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/termvectors/TransportMultiTermVectorsActionTests.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; @@ -224,6 +225,10 @@ private static Task createTask() { static class Resolver extends IndexNameExpressionResolver { + Resolver() { + super(new ThreadContext(Settings.EMPTY)); + } + @Override public Index concreteSingleIndex(ClusterState state, IndicesRequest request) { return new Index("index1", randomBase64UUID()); diff --git a/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java b/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java index 80d516709c80b..e122d5bb4c4ba 100644 --- a/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java @@ -52,6 +52,8 @@ import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.plugins.ClusterPlugin; import org.elasticsearch.test.gateway.TestGatewayAllocator; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; import java.util.Collection; @@ -65,6 +67,8 @@ public class ClusterModuleTests extends ModuleTestCase { private ClusterInfoService clusterInfoService = EmptyClusterInfoService.INSTANCE; private ClusterService clusterService = new ClusterService(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null); + private ThreadPool threadPool = new TestThreadPool(this.getClass().getSimpleName() + "ThreadPool"); + static class FakeAllocationDecider extends AllocationDecider { protected FakeAllocationDecider() { } @@ -119,7 +123,7 @@ public void testRegisterAllocationDeciderDuplicate() { public Collection createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) { return Collections.singletonList(new EnableAllocationDecider(settings, clusterSettings)); } - }), clusterInfoService)); + }), clusterInfoService, threadPool)); assertEquals(e.getMessage(), "Cannot specify allocation decider [" + EnableAllocationDecider.class.getName() + "] twice"); } @@ -131,7 +135,7 @@ public void testRegisterAllocationDecider() { public Collection createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) { return Collections.singletonList(new FakeAllocationDecider()); } - }), clusterInfoService); + }), clusterInfoService, threadPool); assertTrue(module.deciderList.stream().anyMatch(d -> d.getClass().equals(FakeAllocationDecider.class))); } @@ -143,7 +147,7 @@ public Map> getShardsAllocators(Settings setti return Collections.singletonMap(name, supplier); } } - ), clusterInfoService); + ), clusterInfoService, threadPool); } public void testRegisterShardsAllocator() { @@ -161,7 +165,7 @@ public void testRegisterShardsAllocatorAlreadyRegistered() { public void testUnknownShardsAllocator() { Settings settings = Settings.builder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING.getKey(), "dne").build(); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> - new ClusterModule(settings, clusterService, Collections.emptyList(), clusterInfoService)); + new ClusterModule(settings, clusterService, Collections.emptyList(), clusterInfoService, threadPool)); assertEquals("Unknown ShardsAllocator [dne]", e.getMessage()); } @@ -204,13 +208,14 @@ public void testAllocationDeciderOrder() { public void testRejectsReservedExistingShardsAllocatorName() { final ClusterModule clusterModule = new ClusterModule(Settings.EMPTY, clusterService, - List.of(existingShardsAllocatorPlugin(GatewayAllocator.ALLOCATOR_NAME)), clusterInfoService); + List.of(existingShardsAllocatorPlugin(GatewayAllocator.ALLOCATOR_NAME)), clusterInfoService, threadPool); expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator())); } public void testRejectsDuplicateExistingShardsAllocatorName() { final ClusterModule clusterModule = new ClusterModule(Settings.EMPTY, clusterService, - List.of(existingShardsAllocatorPlugin("duplicate"), existingShardsAllocatorPlugin("duplicate")), clusterInfoService); + List.of(existingShardsAllocatorPlugin("duplicate"), existingShardsAllocatorPlugin("duplicate")), + clusterInfoService, threadPool); expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator())); } diff --git a/server/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java b/server/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java index f67e74c29f39b..67e4c1e3f2cd0 100644 --- a/server/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java @@ -48,6 +48,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.gateway.TestGatewayAllocator; @@ -79,7 +80,8 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo; public class ClusterStateHealthTests extends ESTestCase { - private final IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + private final IndexNameExpressionResolver indexNameExpressionResolver = + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); private static ThreadPool threadPool; diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java index ee1a93646cfe3..fbef1a216c8de 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java @@ -43,8 +43,8 @@ public class DateMathExpressionResolverTests extends ESTestCase { private final DateMathExpressionResolver expressionResolver = new DateMathExpressionResolver(); private final Context context = new Context( - ClusterState.builder(new ClusterName("_name")).build(), IndicesOptions.strictExpand() - ); + ClusterState.builder(new ClusterName("_name")).build(), IndicesOptions.strictExpand(), + false); public void testNormal() throws Exception { int numIndexExpressions = randomIntBetween(1, 9); @@ -146,7 +146,7 @@ public void testExpression_CustomTimeZoneInIndexName() throws Exception { // rounding to today 00:00 now = DateTime.now(UTC).withHourOfDay(0).withMinuteOfHour(0).withSecondOfMinute(0); } - Context context = new Context(this.context.getState(), this.context.getOptions(), now.getMillis()); + Context context = new Context(this.context.getState(), this.context.getOptions(), now.getMillis(), false); List results = expressionResolver.resolve(context, Arrays.asList("<.marvel-{now/d{yyyy.MM.dd|" + timeZone.getID() + "}}>")); assertThat(results.size(), equalTo(1)); logger.info("timezone: [{}], now [{}], name: [{}]", timeZone, now, results.get(0)); diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverAliasIterationTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverAliasIterationTests.java index 13d3cfd6cea95..5fd417c1eec50 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverAliasIterationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverAliasIterationTests.java @@ -19,10 +19,13 @@ package org.elasticsearch.cluster.metadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; + public class IndexNameExpressionResolverAliasIterationTests extends IndexNameExpressionResolverTests { - protected IndexNameExpressionResolver getIndexNameExpressionResolver() { - return new IndexNameExpressionResolver() { + protected IndexNameExpressionResolver createIndexNameExpressionResolver() { + return new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)) { @Override boolean iterateIndexAliases(int indexAliasesSize, int resolvedExpressionsSize) { return true; diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverExpressionsIterationTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverExpressionsIterationTests.java index 00d46aad0e8cd..79760be1fecbc 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverExpressionsIterationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverExpressionsIterationTests.java @@ -19,10 +19,13 @@ package org.elasticsearch.cluster.metadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; + public class IndexNameExpressionResolverExpressionsIterationTests extends IndexNameExpressionResolverTests { - protected IndexNameExpressionResolver getIndexNameExpressionResolver() { - return new IndexNameExpressionResolver() { + protected IndexNameExpressionResolver createIndexNameExpressionResolver() { + return new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)) { @Override boolean iterateIndexAliases(int indexAliasesSize, int resolvedExpressionsSize) { return false; 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 8dcd79d781ad1..88de93ad87dc4 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.cluster.ClusterName; @@ -33,6 +34,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata.State; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexSettings; @@ -48,6 +50,7 @@ import java.util.List; import java.util.Set; import java.util.function.Function; +import java.util.stream.Collectors; import static org.elasticsearch.cluster.DataStreamTestHelper.createBackingIndex; import static org.elasticsearch.cluster.DataStreamTestHelper.createTimestampField; @@ -67,15 +70,21 @@ public class IndexNameExpressionResolverTests extends ESTestCase { private IndexNameExpressionResolver indexNameExpressionResolver; + private ThreadContext threadContext; - protected IndexNameExpressionResolver getIndexNameExpressionResolver() { - return new IndexNameExpressionResolver(); + private ThreadContext createThreadContext() { + return new ThreadContext(Settings.EMPTY); + } + + protected IndexNameExpressionResolver createIndexNameExpressionResolver(ThreadContext threadContext) { + return new IndexNameExpressionResolver(threadContext); } @Override public void setUp() throws Exception { super.setUp(); - indexNameExpressionResolver = getIndexNameExpressionResolver(); + threadContext = createThreadContext(); + indexNameExpressionResolver = createIndexNameExpressionResolver(threadContext); } public void testIndexOptionsStrict() { @@ -89,7 +98,7 @@ public void testIndexOptionsStrict() { IndicesOptions[] indicesOptions = new IndicesOptions[]{ IndicesOptions.strictExpandOpen(), IndicesOptions.strictExpand()}; for (IndicesOptions options : indicesOptions) { - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options, false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, "foo"); assertEquals(1, results.length); assertEquals("foo", results[0]); @@ -138,26 +147,27 @@ public void testIndexOptionsStrict() { assertEquals("foo", results[0]); } - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen(), false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(3, results.length); results = indexNameExpressionResolver.concreteIndexNames(context, (String[])null); assertEquals(3, results.length); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpand()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpand(), false); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(4, results.length); results = indexNameExpressionResolver.concreteIndexNames(context, (String[])null); assertEquals(4, results.length); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen(), false); results = indexNameExpressionResolver.concreteIndexNames(context, "foofoo*"); assertEquals(3, results.length); assertThat(results, arrayContainingInAnyOrder("foo", "foobar", "foofoo")); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpand()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpand(), false); results = indexNameExpressionResolver.concreteIndexNames(context, "foofoo*"); assertEquals(4, results.length); assertThat(results, arrayContainingInAnyOrder("foo", "foobar", "foofoo", "foofoo-closed")); @@ -173,7 +183,7 @@ public void testIndexOptionsLenient() { IndicesOptions[] indicesOptions = new IndicesOptions[]{IndicesOptions.lenientExpandOpen(), IndicesOptions.lenientExpand()}; for (IndicesOptions options : indicesOptions) { - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options, false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, "foo"); assertEquals(1, results.length); assertEquals("foo", results[0]); @@ -210,20 +220,21 @@ public void testIndexOptionsLenient() { assertEquals("foo", results[0]); } - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(3, results.length); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpand()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpand(), false); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(Arrays.toString(results), 4, results.length); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); results = indexNameExpressionResolver.concreteIndexNames(context, "foofoo*"); assertEquals(3, results.length); assertThat(results, arrayContainingInAnyOrder("foo", "foobar", "foofoo")); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpand()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpand(), false); results = indexNameExpressionResolver.concreteIndexNames(context, "foofoo*"); assertEquals(4, results.length); assertThat(results, arrayContainingInAnyOrder("foo", "foobar", "foofoo", "foofoo-closed")); @@ -242,7 +253,7 @@ public void testIndexOptionsAllowUnavailableDisallowEmpty() { IndicesOptions[] indicesOptions = new IndicesOptions[]{expandOpen, expand}; for (IndicesOptions options : indicesOptions) { - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options, false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, "foo"); assertEquals(1, results.length); assertEquals("foo", results[0]); @@ -264,11 +275,11 @@ public void testIndexOptionsAllowUnavailableDisallowEmpty() { } } - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, expandOpen); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, expandOpen, false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(3, results.length); - context = new IndexNameExpressionResolver.Context(state, expand); + context = new IndexNameExpressionResolver.Context(state, expand, false); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(4, results.length); } @@ -286,7 +297,7 @@ public void testIndexOptionsWildcardExpansion() { // Only closed IndicesOptions options = IndicesOptions.fromOptions(false, true, false, true, false); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options, false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(1, results.length); assertEquals("foo", results[0]); @@ -311,7 +322,7 @@ public void testIndexOptionsWildcardExpansion() { // Only open options = IndicesOptions.fromOptions(false, true, true, false, false); - context = new IndexNameExpressionResolver.Context(state, options); + context = new IndexNameExpressionResolver.Context(state, options, false); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(2, results.length); assertThat(results, arrayContainingInAnyOrder("bar", "foobar")); @@ -335,7 +346,7 @@ public void testIndexOptionsWildcardExpansion() { // Open and closed options = IndicesOptions.fromOptions(false, true, true, true, false); - context = new IndexNameExpressionResolver.Context(state, options); + context = new IndexNameExpressionResolver.Context(state, options, false); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(3, results.length); assertThat(results, arrayContainingInAnyOrder("bar", "foobar", "foo")); @@ -374,7 +385,7 @@ public void testIndexOptionsWildcardExpansion() { // open closed and hidden options = IndicesOptions.fromOptions(false, true, true, true, true); - context = new IndexNameExpressionResolver.Context(state, options); + context = new IndexNameExpressionResolver.Context(state, options, false); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(7, results.length); assertThat(results, arrayContainingInAnyOrder("bar", "foobar", "foo", "hidden", "hidden-closed", ".hidden", ".hidden-closed")); @@ -416,7 +427,7 @@ public void testIndexOptionsWildcardExpansion() { // open and hidden options = IndicesOptions.fromOptions(false, true, true, false, true); - context = new IndexNameExpressionResolver.Context(state, options); + context = new IndexNameExpressionResolver.Context(state, options, false); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(4, results.length); assertThat(results, arrayContainingInAnyOrder("bar", "foobar", "hidden", ".hidden")); @@ -435,7 +446,7 @@ public void testIndexOptionsWildcardExpansion() { // closed and hidden options = IndicesOptions.fromOptions(false, true, false, true, true); - context = new IndexNameExpressionResolver.Context(state, options); + context = new IndexNameExpressionResolver.Context(state, options, false); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertEquals(3, results.length); assertThat(results, arrayContainingInAnyOrder("foo", "hidden-closed", ".hidden-closed")); @@ -454,7 +465,7 @@ public void testIndexOptionsWildcardExpansion() { // only hidden options = IndicesOptions.fromOptions(false, true, false, false, true); - context = new IndexNameExpressionResolver.Context(state, options); + context = new IndexNameExpressionResolver.Context(state, options, false); results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertThat(results, emptyArray()); @@ -468,7 +479,7 @@ public void testIndexOptionsWildcardExpansion() { assertThat(results, arrayContainingInAnyOrder("hidden-closed")); options = IndicesOptions.fromOptions(false, false, true, true, true); - IndexNameExpressionResolver.Context context2 = new IndexNameExpressionResolver.Context(state, options); + IndexNameExpressionResolver.Context context2 = new IndexNameExpressionResolver.Context(state, options, false); IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, () -> indexNameExpressionResolver.concreteIndexNames(context2, "-*")); assertThat(infe.getResourceId().toString(), equalTo("[-*]")); @@ -485,7 +496,7 @@ public void testIndexOptionsNoExpandWildcards() { //ignore unavailable and allow no indices { IndicesOptions noExpandLenient = IndicesOptions.fromOptions(true, true, false, false, randomBoolean()); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, noExpandLenient); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, noExpandLenient, false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, "baz*"); assertThat(results, emptyArray()); @@ -507,7 +518,7 @@ public void testIndexOptionsNoExpandWildcards() { //ignore unavailable but don't allow no indices { IndicesOptions noExpandDisallowEmpty = IndicesOptions.fromOptions(true, false, false, false, randomBoolean()); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, noExpandDisallowEmpty); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, noExpandDisallowEmpty, false); { IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, @@ -532,7 +543,7 @@ public void testIndexOptionsNoExpandWildcards() { //error on unavailable but allow no indices { IndicesOptions noExpandErrorUnavailable = IndicesOptions.fromOptions(false, true, false, false, randomBoolean()); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, noExpandErrorUnavailable); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, noExpandErrorUnavailable, false); { String[] results = indexNameExpressionResolver.concreteIndexNames(context, "baz*"); assertThat(results, emptyArray()); @@ -558,7 +569,7 @@ public void testIndexOptionsNoExpandWildcards() { //error on both unavailable and no indices { IndicesOptions noExpandStrict = IndicesOptions.fromOptions(false, false, false, false, randomBoolean()); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, noExpandStrict); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, noExpandStrict, false); IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, () -> indexNameExpressionResolver.concreteIndexNames(context, "baz*")); assertThat(infe.getIndex().getName(), equalTo("baz*")); @@ -585,7 +596,7 @@ public void testIndexOptionsSingleIndexNoExpandWildcards() { { IndexNameExpressionResolver.Context context = - new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed()); + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed(), false); IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, () -> indexNameExpressionResolver.concreteIndexNames(context, "baz*")); assertThat(infe.getIndex().getName(), equalTo("baz*")); @@ -593,7 +604,7 @@ public void testIndexOptionsSingleIndexNoExpandWildcards() { { IndexNameExpressionResolver.Context context = - new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed()); + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed(), false); IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, () -> indexNameExpressionResolver.concreteIndexNames(context, "foo", "baz*")); assertThat(infe.getIndex().getName(), equalTo("baz*")); @@ -601,7 +612,7 @@ public void testIndexOptionsSingleIndexNoExpandWildcards() { { IndexNameExpressionResolver.Context context = - new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed()); + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed(), false); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> indexNameExpressionResolver.concreteIndexNames(context, "foofoobar")); assertThat(e.getMessage(), containsString("alias [foofoobar] has more than one index associated with it")); @@ -609,7 +620,7 @@ public void testIndexOptionsSingleIndexNoExpandWildcards() { { IndexNameExpressionResolver.Context context = - new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed()); + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed(), false); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> indexNameExpressionResolver.concreteIndexNames(context, "foo", "foofoobar")); assertThat(e.getMessage(), containsString("alias [foofoobar] has more than one index associated with it")); @@ -617,7 +628,7 @@ public void testIndexOptionsSingleIndexNoExpandWildcards() { { IndexNameExpressionResolver.Context context = - new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed()); + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed(), false); IndexClosedException ince = expectThrows(IndexClosedException.class, () -> indexNameExpressionResolver.concreteIndexNames(context, "foofoo-closed", "foofoobar")); assertThat(ince.getMessage(), equalTo("closed")); @@ -625,7 +636,7 @@ public void testIndexOptionsSingleIndexNoExpandWildcards() { } IndexNameExpressionResolver.Context context = - new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed()); + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictSingleIndexNoExpandForbidClosed(), false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, "foo", "barbaz"); assertEquals(2, results.length); assertThat(results, arrayContainingInAnyOrder("foo", "foofoo")); @@ -635,7 +646,7 @@ public void testIndexOptionsEmptyCluster() { ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(Metadata.builder().build()).build(); IndicesOptions options = IndicesOptions.strictExpandOpen(); - final IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options); + final IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options, false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, Strings.EMPTY_ARRAY); assertThat(results, emptyArray()); @@ -656,7 +667,7 @@ public void testIndexOptionsEmptyCluster() { final IndexNameExpressionResolver.Context context2 = - new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); results = indexNameExpressionResolver.concreteIndexNames(context2, Strings.EMPTY_ARRAY); assertThat(results, emptyArray()); results = indexNameExpressionResolver.concreteIndexNames(context2, "foo"); @@ -667,7 +678,7 @@ public void testIndexOptionsEmptyCluster() { assertThat(results, emptyArray()); final IndexNameExpressionResolver.Context context3 = - new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, false, true, false)); + new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, false, true, false), false); IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, () -> indexNameExpressionResolver.concreteIndexNames(context3, Strings.EMPTY_ARRAY)); assertThat(infe.getResourceId().toString(), equalTo("[_all]")); @@ -692,7 +703,8 @@ public void testConcreteIndicesIgnoreIndicesOneMissingIndex() { .put(indexBuilder("testXXX")) .put(indexBuilder("kuku")); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen(), false); IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, () -> indexNameExpressionResolver.concreteIndexNames(context, "testZZZ")); @@ -704,7 +716,8 @@ public void testConcreteIndicesIgnoreIndicesOneMissingIndexOtherFound() { .put(indexBuilder("testXXX")) .put(indexBuilder("kuku")); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "testXXX", "testZZZ")), equalTo(newHashSet("testXXX"))); @@ -715,7 +728,8 @@ public void testConcreteIndicesIgnoreIndicesAllMissing() { .put(indexBuilder("testXXX")) .put(indexBuilder("kuku")); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen(), false); IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, () -> indexNameExpressionResolver.concreteIndexNames(context, "testMo", "testMahdy")); @@ -727,7 +741,8 @@ public void testConcreteIndicesIgnoreIndicesEmptyRequest() { .put(indexBuilder("testXXX")) .put(indexBuilder("kuku")); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, new String[]{})), equalTo(newHashSet("kuku", "testXXX"))); } @@ -735,7 +750,7 @@ public void testConcreteIndicesNoIndicesErrorMessage() { Metadata.Builder mdBuilder = Metadata.builder(); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, - IndicesOptions.fromOptions(false, false, true, true)); + IndicesOptions.fromOptions(false, false, true, true), false); IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, () -> indexNameExpressionResolver.concreteIndices(context, new String[]{})); assertThat(infe.getMessage(), is("no such index [null] and no indices exist")); @@ -745,7 +760,7 @@ public void testConcreteIndicesNoIndicesErrorMessageNoExpand() { Metadata.Builder mdBuilder = Metadata.builder(); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, - IndicesOptions.fromOptions(false, false, false, false)); + IndicesOptions.fromOptions(false, false, false, false), false); IndexNotFoundException infe = expectThrows(IndexNotFoundException.class, () -> indexNameExpressionResolver.concreteIndices(context, new String[]{})); assertThat(infe.getMessage(), is("no such index [_all] and no indices exist")); @@ -761,16 +776,16 @@ public void testConcreteIndicesWildcardExpansion() { ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); IndexNameExpressionResolver.Context context = - new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, false, false)); + new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, false, false), false); assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "testX*")), equalTo(new HashSet())); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, true, false)); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, true, false), false); assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "testX*")), equalTo(newHashSet("testXXX", "testXXY"))); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, false, true)); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, false, true), false); assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "testX*")), equalTo(newHashSet("testXYY"))); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, true, true)); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, true, true), false); assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "testX*")), equalTo(newHashSet("testXXX", "testXXY", "testXYY"))); } @@ -788,7 +803,7 @@ public void testConcreteIndicesWildcardWithNegation() { ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, - IndicesOptions.fromOptions(true, true, true, true)); + IndicesOptions.fromOptions(true, true, true, true), false); assertThat(newHashSet(indexNameExpressionResolver.concreteIndexNames(context, "testX*")), equalTo(newHashSet("testXXX", "testXXY", "testXYY"))); @@ -1076,7 +1091,7 @@ public void testConcreteIndicesAllPatternRandom() { { ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(Metadata.builder().build()).build(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, indicesOptions); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, indicesOptions, false); // with no indices, asking for all indices should return empty list or exception, depending on indices options if (indicesOptions.allowNoIndices()) { @@ -1095,7 +1110,7 @@ public void testConcreteIndicesAllPatternRandom() { .put(indexBuilder("bbb").state(State.OPEN).putAlias(AliasMetadata.builder("bbb_alias1"))) .put(indexBuilder("ccc").state(State.CLOSE).putAlias(AliasMetadata.builder("ccc_alias1"))); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, indicesOptions); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, indicesOptions, false); if (indicesOptions.expandWildcardsOpen() || indicesOptions.expandWildcardsClosed() || indicesOptions.allowNoIndices()) { String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(context, allIndices); assertThat(concreteIndices, notNullValue()); @@ -1125,7 +1140,7 @@ public void testConcreteIndicesWildcardNoMatch() { .put(indexBuilder("bbb").state(State.OPEN).putAlias(AliasMetadata.builder("bbb_alias1"))) .put(indexBuilder("ccc").state(State.CLOSE).putAlias(AliasMetadata.builder("ccc_alias1"))); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, indicesOptions); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, indicesOptions, false); // asking for non existing wildcard pattern should return empty list or exception if (indicesOptions.allowNoIndices()) { @@ -1254,20 +1269,20 @@ public void testIndexOptionsFailClosedIndicesAndAliases() { ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); IndexNameExpressionResolver.Context contextICE = - new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpenAndForbidClosed()); + new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpenAndForbidClosed(), false); expectThrows(IndexClosedException.class, () -> indexNameExpressionResolver.concreteIndexNames(contextICE, "foo1-closed")); expectThrows(IndexClosedException.class, () -> indexNameExpressionResolver.concreteIndexNames(contextICE, "foobar1-closed")); IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, contextICE.getOptions().allowNoIndices(), contextICE.getOptions().expandWildcardsOpen(), - contextICE.getOptions().expandWildcardsClosed(), contextICE.getOptions())); + contextICE.getOptions().expandWildcardsClosed(), contextICE.getOptions()), false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, "foo1-closed"); assertThat(results, emptyArray()); results = indexNameExpressionResolver.concreteIndexNames(context, "foobar1-closed"); assertThat(results, emptyArray()); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); results = indexNameExpressionResolver.concreteIndexNames(context, "foo1-closed"); assertThat(results, arrayWithSize(1)); assertThat(results, arrayContaining("foo1-closed")); @@ -1277,7 +1292,7 @@ public void testIndexOptionsFailClosedIndicesAndAliases() { assertThat(results, arrayContaining("foo1-closed")); // testing an alias pointing to three indices: - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpenAndForbidClosed()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpenAndForbidClosed(), false); try { indexNameExpressionResolver.concreteIndexNames(context, "foobar2-closed"); fail("foo2-closed should be closed, but it is open"); @@ -1287,12 +1302,12 @@ public void testIndexOptionsFailClosedIndicesAndAliases() { context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, context.getOptions().allowNoIndices(), context.getOptions().expandWildcardsOpen(), - context.getOptions().expandWildcardsClosed(), context.getOptions())); + context.getOptions().expandWildcardsClosed(), context.getOptions()), false); results = indexNameExpressionResolver.concreteIndexNames(context, "foobar2-closed"); assertThat(results, arrayWithSize(1)); assertThat(results, arrayContaining("foo3")); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); results = indexNameExpressionResolver.concreteIndexNames(context, "foobar2-closed"); assertThat(results, arrayWithSize(3)); assertThat(results, arrayContainingInAnyOrder("foo1-closed", "foo2-closed", "foo3")); @@ -1305,7 +1320,7 @@ public void testDedupConcreteIndices() { IndicesOptions[] indicesOptions = new IndicesOptions[]{ IndicesOptions.strictExpandOpen(), IndicesOptions.strictExpand(), IndicesOptions.lenientExpandOpen(), IndicesOptions.strictExpandOpenAndForbidClosed()}; for (IndicesOptions options : indicesOptions) { - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options); + IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, options, false); String[] results = indexNameExpressionResolver.concreteIndexNames(context, "index1", "index1", "alias1"); assertThat(results, equalTo(new String[]{"index1"})); } @@ -1325,11 +1340,12 @@ public void testFilterClosedIndicesOnAliases() { .put(indexBuilder("test-1").state(IndexMetadata.State.CLOSE).putAlias(AliasMetadata.builder("alias-1"))); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); String[] strings = indexNameExpressionResolver.concreteIndexNames(context, "alias-*"); assertArrayEquals(new String[] {"test-0"}, strings); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen()); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen(), false); strings = indexNameExpressionResolver.concreteIndexNames(context, "alias-*"); assertArrayEquals(new String[] {"test-0"}, strings); @@ -1739,7 +1755,8 @@ public void testIndicesAliasesRequestTargetDataStreams() { public void testInvalidIndex() { Metadata.Builder mdBuilder = Metadata.builder().put(indexBuilder("test")); ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); InvalidIndexNameException iine = expectThrows(InvalidIndexNameException.class, () -> indexNameExpressionResolver.concreteIndexNames(context, "_foo")); @@ -1810,6 +1827,95 @@ public void testIgnoreThrottled() { } } + public void testSystemIndexResolutionWhenAllowed() { + Settings settings = Settings.builder().build(); + Metadata.Builder mdBuilder = Metadata.builder() + .put(indexBuilder(".ml-meta", settings).state(State.OPEN).system(true)) + .put(indexBuilder(".watches", settings).state(State.OPEN).system(true)) + .put(indexBuilder(".ml-stuff", settings).state(State.OPEN).system(true)) + .put(indexBuilder("some-other-index").state(State.OPEN)); + ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); + + // Single name + { + SearchRequest request = new SearchRequest(".ml-meta"); + + List indexNames = Arrays + .stream(indexNameExpressionResolver.concreteIndices(state, request)) + .map(i -> i.getName()) + .collect(Collectors.toList()); + assertThat(indexNames, containsInAnyOrder(".ml-meta")); + } + + // Wildcard that should match multiple + { + SearchRequest request = new SearchRequest(".ml-*"); + + List indexNames = Arrays + .stream(indexNameExpressionResolver.concreteIndices(state, request)) + .map(i -> i.getName()) + .collect(Collectors.toList()); + assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); + } + + // Wildcard that just matches one + { + SearchRequest request = new SearchRequest(".w*"); + + List indexNames = Arrays + .stream(indexNameExpressionResolver.concreteIndices(state, request)) + .map(i -> i.getName()) + .collect(Collectors.toList()); + assertThat(indexNames, containsInAnyOrder(".watches")); + } + } + + public void testSystemIndexResolutionBlocked() { + // Set up the thread context to disallow system index access + threadContext.putHeader("_from_rest", "true"); + + Settings settings = Settings.builder().build(); + Metadata.Builder mdBuilder = Metadata.builder() + .put(indexBuilder(".ml-meta", settings).state(State.OPEN).system(true)) + .put(indexBuilder(".watches", settings).state(State.OPEN).system(true)) + .put(indexBuilder(".ml-stuff", settings).state(State.OPEN).system(true)) + .put(indexBuilder("some-other-index").state(State.OPEN)); + ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); + + // Wildcard that might match multiple + { + SearchRequest request = new SearchRequest(".ml-*"); + + List indexNames = Arrays + .stream(indexNameExpressionResolver.concreteIndices(state, request)) + .map(i -> i.getName()) + .collect(Collectors.toList()); + assertThat(indexNames, empty()); + } + + // Wilcard that might match a single index + { + SearchRequest request = new SearchRequest(".w*"); + + List indexNames = Arrays + .stream(indexNameExpressionResolver.concreteIndices(state, request)) + .map(i -> i.getName()) + .collect(Collectors.toList()); + assertThat(indexNames, empty()); + } + + // A specific index name + { + SearchRequest request = new SearchRequest(".ml-meta"); + + List indexNames = Arrays + .stream(indexNameExpressionResolver.concreteIndices(state, request)) + .map(i -> i.getName()) + .collect(Collectors.toList()); + assertThat(indexNames, empty()); + } + } + public void testDataStreams() { final String dataStreamName = "my-data-stream"; IndexMetadata index1 = createBackingIndex(dataStreamName, 1).build(); diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/WildcardExpressionResolverTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/WildcardExpressionResolverTests.java index b68db18a1cec2..b0fd3c2523280 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/WildcardExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/WildcardExpressionResolverTests.java @@ -48,7 +48,8 @@ public void testConvertWildcardsJustIndicesTests() { ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); IndexNameExpressionResolver.WildcardExpressionResolver resolver = new IndexNameExpressionResolver.WildcardExpressionResolver(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); assertThat(newHashSet(resolver.resolve(context, Collections.singletonList("testXXX"))), equalTo(newHashSet("testXXX"))); assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testXXX", "testYYY"))), equalTo(newHashSet("testXXX", "testYYY"))); assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testXXX", "ku*"))), equalTo(newHashSet("testXXX", "kuku"))); @@ -76,7 +77,8 @@ public void testConvertWildcardsTests() { ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); IndexNameExpressionResolver.WildcardExpressionResolver resolver = new IndexNameExpressionResolver.WildcardExpressionResolver(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testYY*", "alias*"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY"))); assertThat(newHashSet(resolver.resolve(context, Collections.singletonList("-kuku"))), equalTo(newHashSet("-kuku"))); @@ -99,12 +101,12 @@ public void testConvertWildcardsOpenClosedIndicesTests() { IndexNameExpressionResolver.WildcardExpressionResolver resolver = new IndexNameExpressionResolver.WildcardExpressionResolver(); IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, - IndicesOptions.fromOptions(true, true, true, true)); + IndicesOptions.fromOptions(true, true, true, true), false); assertThat(newHashSet(resolver.resolve(context, Collections.singletonList("testX*"))), equalTo(newHashSet("testXXX", "testXXY", "testXYY"))); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, false, true)); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, false, true), false); assertThat(newHashSet(resolver.resolve(context, Collections.singletonList("testX*"))), equalTo(newHashSet("testXYY"))); - context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, true, false)); + context = new IndexNameExpressionResolver.Context(state, IndicesOptions.fromOptions(true, true, true, false), false); assertThat(newHashSet(resolver.resolve(context, Collections.singletonList("testX*"))), equalTo(newHashSet("testXXX", "testXXY"))); } @@ -121,7 +123,8 @@ public void testMultipleWildcards() { ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); IndexNameExpressionResolver.WildcardExpressionResolver resolver = new IndexNameExpressionResolver.WildcardExpressionResolver(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); assertThat(newHashSet(resolver.resolve(context, Collections.singletonList("test*X*"))), equalTo(newHashSet("testXXX", "testXXY", "testXYY"))); assertThat(newHashSet(resolver.resolve(context, Collections.singletonList("test*X*Y"))), equalTo(newHashSet("testXXY", "testXYY"))); @@ -140,7 +143,8 @@ public void testAll() { ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); IndexNameExpressionResolver.WildcardExpressionResolver resolver = new IndexNameExpressionResolver.WildcardExpressionResolver(); - IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen()); + IndexNameExpressionResolver.Context context = + new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen(), false); assertThat(newHashSet(resolver.resolve(context, Collections.singletonList("_all"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY"))); } @@ -158,15 +162,15 @@ public void testResolveAliases() { IndicesOptions indicesAndAliasesOptions = IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, false, true, false, false, false); IndexNameExpressionResolver.Context indicesAndAliasesContext = - new IndexNameExpressionResolver.Context(state, indicesAndAliasesOptions); + new IndexNameExpressionResolver.Context(state, indicesAndAliasesOptions, false); // ignoreAliases option is set, WildcardExpressionResolver throws error when IndicesOptions skipAliasesIndicesOptions = IndicesOptions.fromOptions(true, true, true, false, true, false, true, false); IndexNameExpressionResolver.Context skipAliasesLenientContext = - new IndexNameExpressionResolver.Context(state, skipAliasesIndicesOptions); + new IndexNameExpressionResolver.Context(state, skipAliasesIndicesOptions, false); // ignoreAliases option is set, WildcardExpressionResolver resolves the provided expressions only against the defined indices IndicesOptions errorOnAliasIndicesOptions = IndicesOptions.fromOptions(false, false, true, false, true, false, true, false); IndexNameExpressionResolver.Context skipAliasesStrictContext = - new IndexNameExpressionResolver.Context(state, errorOnAliasIndicesOptions); + new IndexNameExpressionResolver.Context(state, errorOnAliasIndicesOptions, false); { List indices = resolver.resolve(indicesAndAliasesContext, Collections.singletonList("foo_a*")); @@ -232,7 +236,7 @@ public void testResolveDataStreams() { IndicesOptions indicesAndAliasesOptions = IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, false, true, false, false, false); IndexNameExpressionResolver.Context indicesAndAliasesContext = - new IndexNameExpressionResolver.Context(state, indicesAndAliasesOptions); + new IndexNameExpressionResolver.Context(state, indicesAndAliasesOptions, false); // data streams are not included but expression matches the data stream List indices = resolver.resolve(indicesAndAliasesContext, Collections.singletonList("foo_*")); @@ -247,7 +251,7 @@ public void testResolveDataStreams() { IndicesOptions indicesAndAliasesOptions = IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, false, true, false, false, false); IndexNameExpressionResolver.Context indicesAliasesAndDataStreamsContext = new IndexNameExpressionResolver.Context(state, - indicesAndAliasesOptions, false, false, true); + indicesAndAliasesOptions, false, false, true, false); // data stream's corresponding backing indices are resolved List indices = resolver.resolve(indicesAliasesAndDataStreamsContext, Collections.singletonList("foo_*")); @@ -264,7 +268,7 @@ public void testResolveDataStreams() { IndicesOptions indicesAliasesAndExpandHiddenOptions = IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, false, true, true, false, false, false); IndexNameExpressionResolver.Context indicesAliasesDataStreamsAndHiddenIndices = new IndexNameExpressionResolver.Context(state, - indicesAliasesAndExpandHiddenOptions, false, false, true); + indicesAliasesAndExpandHiddenOptions, false, false, true, false); // data stream's corresponding backing indices are resolved List indices = resolver.resolve(indicesAliasesDataStreamsAndHiddenIndices, Collections.singletonList("foo_*")); @@ -290,12 +294,12 @@ public void testMatchesConcreteIndicesWildcardAndAliases() { // expressions against the defined indices and aliases IndicesOptions indicesAndAliasesOptions = IndicesOptions.fromOptions(false, false, true, false, true, false, false, false); IndexNameExpressionResolver.Context indicesAndAliasesContext = - new IndexNameExpressionResolver.Context(state, indicesAndAliasesOptions); + new IndexNameExpressionResolver.Context(state, indicesAndAliasesOptions, false); // ignoreAliases option is set, WildcardExpressionResolver resolves the provided expressions // only against the defined indices IndicesOptions onlyIndicesOptions = IndicesOptions.fromOptions(false, false, true, false, true, false, true, false); - IndexNameExpressionResolver.Context onlyIndicesContext = new IndexNameExpressionResolver.Context(state, onlyIndicesOptions); + IndexNameExpressionResolver.Context onlyIndicesContext = new IndexNameExpressionResolver.Context(state, onlyIndicesOptions, false); { Set matches = IndexNameExpressionResolver.WildcardExpressionResolver.matches(indicesAndAliasesContext, diff --git a/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 9447ac6e43342..58d50dde6d044 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -48,6 +48,7 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; @@ -180,7 +181,7 @@ public void testWrapperIsBound() throws IOException { engineFactory, Collections.emptyMap(), () -> true, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), Collections.emptyMap()); module.setReaderWrapper(s -> new Wrapper()); @@ -202,7 +203,7 @@ public void testRegisterIndexStore() throws IOException { final Map indexStoreFactories = singletonMap( "foo_store", new FooFunction()); final IndexModule module = new IndexModule(indexSettings, emptyAnalysisRegistry, new InternalEngineFactory(), indexStoreFactories, - () -> true, new IndexNameExpressionResolver(), Collections.emptyMap()); + () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), Collections.emptyMap()); final IndexService indexService = newIndexService(module); assertThat(indexService.getDirectoryFactory(), instanceOf(FooFunction.class)); @@ -517,7 +518,7 @@ public void testRegisterCustomRecoveryStateFactory() throws IOException { new InternalEngineFactory(), Collections.emptyMap(), () -> true, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), recoveryStateFactories); final IndexService indexService = newIndexService(module); @@ -538,7 +539,7 @@ private ShardRouting createInitializedShardRouting() { private static IndexModule createIndexModule(IndexSettings indexSettings, AnalysisRegistry emptyAnalysisRegistry) { return new IndexModule(indexSettings, emptyAnalysisRegistry, new InternalEngineFactory(), Collections.emptyMap(), () -> true, - new IndexNameExpressionResolver(), Collections.emptyMap()); + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), Collections.emptyMap()); } class CustomQueryCache implements QueryCache { diff --git a/server/src/test/java/org/elasticsearch/index/query/SearchIndexNameMatcherTests.java b/server/src/test/java/org/elasticsearch/index/query/SearchIndexNameMatcherTests.java index eacb0f641404f..3a3a60287c89c 100644 --- a/server/src/test/java/org/elasticsearch/index/query/SearchIndexNameMatcherTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/SearchIndexNameMatcherTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.test.ESTestCase; import org.junit.Before; @@ -49,8 +50,10 @@ public void setUpMatchers() { ClusterService clusterService = mock(ClusterService.class); when(clusterService.state()).thenReturn(state); - matcher = new SearchIndexNameMatcher("index1", "", clusterService, new IndexNameExpressionResolver()); - remoteMatcher = new SearchIndexNameMatcher("index1", "cluster", clusterService, new IndexNameExpressionResolver()); + matcher = new SearchIndexNameMatcher("index1", "", clusterService, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); + remoteMatcher = new SearchIndexNameMatcher("index1", "cluster", clusterService, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); } private static IndexMetadata.Builder indexBuilder(String index) { diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index fffc0aa24148a..1aadee9856b7c 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -82,6 +82,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; @@ -153,7 +154,7 @@ public ClusterStateChanges(NamedXContentRegistry xContentRegistry, ThreadPool th shardStartedClusterStateTaskExecutor = new ShardStateAction.ShardStartedClusterStateTaskExecutor(allocationService, null, logger); ActionFilters actionFilters = new ActionFilters(Collections.emptySet()); - IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); DestructiveOperations destructiveOperations = new DestructiveOperations(SETTINGS, clusterSettings); Environment environment = TestEnvironment.newEnvironment(SETTINGS); Transport transport = mock(Transport.class); // it's not used diff --git a/server/src/test/java/org/elasticsearch/rest/BaseRestHandlerTests.java b/server/src/test/java/org/elasticsearch/rest/BaseRestHandlerTests.java index c412af527507f..f5a44e40c1057 100644 --- a/server/src/test/java/org/elasticsearch/rest/BaseRestHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/BaseRestHandlerTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Table; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; @@ -29,6 +30,8 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.rest.FakeRestChannel; import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.util.Collections; @@ -39,9 +42,24 @@ import static org.hamcrest.core.StringContains.containsString; import static org.hamcrest.object.HasToString.hasToString; -import static org.mockito.Mockito.mock; public class BaseRestHandlerTests extends ESTestCase { + private NodeClient mockClient; + private ThreadPool threadPool; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool(this.getClass().getSimpleName() + "ThreadPool"); + mockClient = new NodeClient(Settings.EMPTY, threadPool); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + mockClient.close(); + } public void testOneUnconsumedParameters() throws Exception { final AtomicBoolean executed = new AtomicBoolean(); @@ -69,7 +87,7 @@ public List routes() { RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withParams(params).build(); RestChannel channel = new FakeRestChannel(request, randomBoolean(), 1); final IllegalArgumentException e = - expectThrows(IllegalArgumentException.class, () -> handler.handleRequest(request, channel, mock(NodeClient.class))); + expectThrows(IllegalArgumentException.class, () -> handler.handleRequest(request, channel, mockClient)); assertThat(e, hasToString(containsString("request [/] contains unrecognized parameter: [unconsumed]"))); assertFalse(executed.get()); } @@ -101,7 +119,7 @@ public List routes() { RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withParams(params).build(); RestChannel channel = new FakeRestChannel(request, randomBoolean(), 1); final IllegalArgumentException e = - expectThrows(IllegalArgumentException.class, () -> handler.handleRequest(request, channel, mock(NodeClient.class))); + expectThrows(IllegalArgumentException.class, () -> handler.handleRequest(request, channel, mockClient)); assertThat(e, hasToString(containsString("request [/] contains unrecognized parameters: [unconsumed-first], [unconsumed-second]"))); assertFalse(executed.get()); } @@ -145,7 +163,7 @@ public List routes() { RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withParams(params).build(); RestChannel channel = new FakeRestChannel(request, randomBoolean(), 1); final IllegalArgumentException e = - expectThrows(IllegalArgumentException.class, () -> handler.handleRequest(request, channel, mock(NodeClient.class))); + expectThrows(IllegalArgumentException.class, () -> handler.handleRequest(request, channel, mockClient)); assertThat( e, hasToString(containsString( @@ -188,7 +206,7 @@ public List routes() { params.put("response_param", randomAlphaOfLength(8)); RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withParams(params).build(); RestChannel channel = new FakeRestChannel(request, randomBoolean(), 1); - handler.handleRequest(request, channel, mock(NodeClient.class)); + handler.handleRequest(request, channel, mockClient); assertTrue(executed.get()); } @@ -218,7 +236,7 @@ public List routes() { params.put("human", null); RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withParams(params).build(); RestChannel channel = new FakeRestChannel(request, randomBoolean(), 1); - handler.handleRequest(request, channel, mock(NodeClient.class)); + handler.handleRequest(request, channel, mockClient); assertTrue(executed.get()); } @@ -262,7 +280,7 @@ public List routes() { params.put("time", randomAlphaOfLength(8)); RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withParams(params).build(); RestChannel channel = new FakeRestChannel(request, randomBoolean(), 1); - handler.handleRequest(request, channel, mock(NodeClient.class)); + handler.handleRequest(request, channel, mockClient); assertTrue(executed.get()); } @@ -291,7 +309,7 @@ public List routes() { .withContent(new BytesArray(builder.toString()), XContentType.JSON) .build(); final RestChannel channel = new FakeRestChannel(request, randomBoolean(), 1); - handler.handleRequest(request, channel, mock(NodeClient.class)); + handler.handleRequest(request, channel, mockClient); assertTrue(executed.get()); } } @@ -317,7 +335,7 @@ public List routes() { final RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).build(); final RestChannel channel = new FakeRestChannel(request, randomBoolean(), 1); - handler.handleRequest(request, channel, mock(NodeClient.class)); + handler.handleRequest(request, channel, mockClient); assertTrue(executed.get()); } @@ -346,7 +364,7 @@ public List routes() { .build(); final RestChannel channel = new FakeRestChannel(request, randomBoolean(), 1); final IllegalArgumentException e = - expectThrows(IllegalArgumentException.class, () -> handler.handleRequest(request, channel, mock(NodeClient.class))); + expectThrows(IllegalArgumentException.class, () -> handler.handleRequest(request, channel, mockClient)); assertThat(e, hasToString(containsString("request [GET /] does not support having a body"))); assertFalse(executed.get()); } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 329a797c7f874..92f68cb7ffb52 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -65,6 +65,7 @@ import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresAction; import org.elasticsearch.action.admin.indices.shards.TransportIndicesShardStoresAction; import org.elasticsearch.action.bulk.BulkAction; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.IndexingPressure; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; @@ -1474,7 +1475,8 @@ public void onFailure(final Exception e) { }, a -> node, null, emptySet() ); - final IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + final IndexNameExpressionResolver indexNameExpressionResolver = + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); repositoriesService = new RepositoriesService( settings, clusterService, transportService, Collections.singletonMap(FsRepository.TYPE, getRepoFactory(environment)), emptyMap(), threadPool diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index a772d33d685dd..d7c0adf2540c5 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -645,6 +645,7 @@ protected static void wipeAllIndices() throws IOException { try { final Request deleteRequest = new Request("DELETE", "*"); deleteRequest.addParameter("expand_wildcards", "open,closed" + (includeHidden ? ",hidden" : "")); + deleteRequest.addParameter("allow_system_index_access", "true"); final Response response = adminClient().performRequest(deleteRequest); try (InputStream is = response.getEntity().getContent()) { assertTrue((boolean) XContentHelper.convertToMap(XContentType.JSON.xContent(), is, true).get("acknowledged")); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/GenerateSnapshotNameStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/GenerateSnapshotNameStep.java index 1f35c8d56645d..68629927139ae 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/GenerateSnapshotNameStep.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/GenerateSnapshotNameStep.java @@ -132,7 +132,7 @@ public ResolverContext() { } public ResolverContext(long startTime) { - super(null, null, startTime, false, false, false, false); + super(null, null, startTime, false, false, false, false, false); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java index 6b969a5ed1763..54b2a10416c01 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java @@ -99,7 +99,7 @@ public static void createIndexAndAliasIfNecessary(Client client, // The initial index name must be suitable for rollover functionality. String firstConcreteIndex = indexPatternPrefix + "-000001"; String[] concreteIndexNames = - resolver.concreteIndexNames(clusterState, IndicesOptions.lenientExpandOpen(), indexPattern); + resolver.concreteIndexNames(clusterState, IndicesOptions.lenientExpandOpen(), indexPattern); //G-> THIS GONNA BREAK Optional indexPointedByCurrentWriteAlias = clusterState.getMetadata().hasAlias(alias) ? clusterState.getMetadata().getIndicesLookup().get(alias).getIndices().stream().findFirst() : Optional.empty(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/common/validation/SourceDestValidatorTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/common/validation/SourceDestValidatorTests.java index 74d08082ad838..807c3cc506d2c 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/common/validation/SourceDestValidatorTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/common/validation/SourceDestValidatorTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.license.License; import org.elasticsearch.license.RemoteClusterLicenseChecker; import org.elasticsearch.license.XPackLicenseState; @@ -91,7 +92,7 @@ public class SourceDestValidatorTests extends ESTestCase { private final TransportService transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool); private final RemoteClusterService remoteClusterService = transportService.getRemoteClusterService(); private final SourceDestValidator simpleNonRemoteValidator = new SourceDestValidator( - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), remoteClusterService, null, "node_id", @@ -571,7 +572,7 @@ public void testRemoteSourceBasic() throws InterruptedException { Context context = spy( new SourceDestValidator.Context( CLUSTER_STATE, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), remoteClusterService, remoteClusterLicenseCheckerBasic, new String[] { REMOTE_BASIC + ":" + "SOURCE_1" }, @@ -595,7 +596,7 @@ public void testRemoteSourcePlatinum() throws InterruptedException { final Context context = spy( new SourceDestValidator.Context( CLUSTER_STATE, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), remoteClusterService, new RemoteClusterLicenseChecker(clientWithBasicLicense, operationMode -> XPackLicenseState.isAllowedByOperationMode(operationMode, License.OperationMode.PLATINUM)), @@ -625,7 +626,7 @@ public void testRemoteSourcePlatinum() throws InterruptedException { final Context context2 = spy( new SourceDestValidator.Context( CLUSTER_STATE, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), remoteClusterService, new RemoteClusterLicenseChecker(clientWithPlatinumLicense, operationMode -> XPackLicenseState.isAllowedByOperationMode(operationMode, License.OperationMode.PLATINUM)), @@ -646,7 +647,7 @@ public void testRemoteSourcePlatinum() throws InterruptedException { final Context context3 = spy( new SourceDestValidator.Context( CLUSTER_STATE, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), remoteClusterService, new RemoteClusterLicenseChecker(clientWithPlatinumLicense, operationMode -> XPackLicenseState.isAllowedByOperationMode(operationMode, License.OperationMode.PLATINUM)), @@ -668,7 +669,7 @@ public void testRemoteSourcePlatinum() throws InterruptedException { final Context context4 = spy( new SourceDestValidator.Context( CLUSTER_STATE, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), remoteClusterService, new RemoteClusterLicenseChecker(clientWithTrialLicense, operationMode -> XPackLicenseState.isAllowedByOperationMode(operationMode, License.OperationMode.PLATINUM)), @@ -692,7 +693,7 @@ public void testRemoteSourceLicenseInActive() throws InterruptedException { final Context context = spy( new SourceDestValidator.Context( CLUSTER_STATE, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), remoteClusterService, new RemoteClusterLicenseChecker(clientWithExpiredBasicLicense, operationMode -> XPackLicenseState.isAllowedByOperationMode(operationMode, License.OperationMode.PLATINUM)), @@ -719,7 +720,7 @@ public void testRemoteSourceDoesNotExist() throws InterruptedException { Context context = spy( new SourceDestValidator.Context( CLUSTER_STATE, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), remoteClusterService, new RemoteClusterLicenseChecker(clientWithExpiredBasicLicense, operationMode -> XPackLicenseState.isAllowedByOperationMode(operationMode, License.OperationMode.PLATINUM)), diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java index 59006f30e75c8..985cb356efc0d 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -78,7 +79,7 @@ public void testFrom() throws IOException { new TransportAddress(TransportAddress.META_ADDRESS, 9300), "test"); ClusterState state = ClusterState.builder(ClusterName.DEFAULT).metadata(metadata).build(); List datafeeds = Collections.singletonList(DatafeedConfigTests.createRandomizedDatafeedConfig("foo")); - IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); + IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); IndicesOptions indicesOptions = IndicesOptions.fromOptions(false, false, true, true); boolean clusterIssueFound = randomBoolean(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAliasTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAliasTests.java index 605704cc2ad9a..a449685188f15 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAliasTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAliasTests.java @@ -248,7 +248,8 @@ public void testIndexNameComparator() { private void createIndexAndAliasIfNecessary(ClusterState clusterState) { MlIndexAndAlias.createIndexAndAliasIfNecessary( - client, clusterState, new IndexNameExpressionResolver(), TEST_INDEX_PREFIX, TEST_INDEX_ALIAS, listener); + client, clusterState, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + TEST_INDEX_PREFIX, TEST_INDEX_ALIAS, listener); } @SuppressWarnings("unchecked") diff --git a/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/action/GetDataStreamsTransportActionTests.java b/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/action/GetDataStreamsTransportActionTests.java index e4c2cb000a15f..9f231d30ae5cd 100644 --- a/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/action/GetDataStreamsTransportActionTests.java +++ b/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/action/GetDataStreamsTransportActionTests.java @@ -11,6 +11,8 @@ import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.action.GetDataStreamAction; @@ -27,7 +29,11 @@ public void testGetDataStream() { final String dataStreamName = "my-data-stream"; ClusterState cs = getClusterStateWithDataStreams(List.of(new Tuple<>(dataStreamName, 1)), List.of()); GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[] { dataStreamName }); - List dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req); + List dataStreams = GetDataStreamsTransportAction.getDataStreams( + cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + req + ); assertThat(dataStreams.size(), equalTo(1)); assertThat(dataStreams.get(0).getName(), equalTo(dataStreamName)); } @@ -40,24 +46,40 @@ public void testGetDataStreamsWithWildcards() { ); GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[] { dataStreamNames[1].substring(0, 5) + "*" }); - List dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req); + List dataStreams = GetDataStreamsTransportAction.getDataStreams( + cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + req + ); assertThat(dataStreams.size(), equalTo(1)); assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1])); req = new GetDataStreamAction.Request(new String[] { "*" }); - dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req); + dataStreams = GetDataStreamsTransportAction.getDataStreams( + cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + req + ); assertThat(dataStreams.size(), equalTo(2)); assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1])); assertThat(dataStreams.get(1).getName(), equalTo(dataStreamNames[0])); req = new GetDataStreamAction.Request((String[]) null); - dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req); + dataStreams = GetDataStreamsTransportAction.getDataStreams( + cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + req + ); assertThat(dataStreams.size(), equalTo(2)); assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1])); assertThat(dataStreams.get(1).getName(), equalTo(dataStreamNames[0])); req = new GetDataStreamAction.Request(new String[] { "matches-none*" }); - dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req); + dataStreams = GetDataStreamsTransportAction.getDataStreams( + cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + req + ); assertThat(dataStreams.size(), equalTo(0)); } @@ -69,25 +91,37 @@ public void testGetDataStreamsWithoutWildcards() { ); GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[] { dataStreamNames[0], dataStreamNames[1] }); - List dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req); + List dataStreams = GetDataStreamsTransportAction.getDataStreams( + cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + req + ); assertThat(dataStreams.size(), equalTo(2)); assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1])); assertThat(dataStreams.get(1).getName(), equalTo(dataStreamNames[0])); req = new GetDataStreamAction.Request(new String[] { dataStreamNames[1] }); - dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req); + dataStreams = GetDataStreamsTransportAction.getDataStreams( + cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + req + ); assertThat(dataStreams.size(), equalTo(1)); assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[1])); req = new GetDataStreamAction.Request(new String[] { dataStreamNames[0] }); - dataStreams = GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req); + dataStreams = GetDataStreamsTransportAction.getDataStreams( + cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + req + ); assertThat(dataStreams.size(), equalTo(1)); assertThat(dataStreams.get(0).getName(), equalTo(dataStreamNames[0])); GetDataStreamAction.Request req2 = new GetDataStreamAction.Request(new String[] { "foo" }); IndexNotFoundException e = expectThrows( IndexNotFoundException.class, - () -> GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req2) + () -> GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), req2) ); assertThat(e.getMessage(), containsString("no such index [foo]")); } @@ -98,7 +132,7 @@ public void testGetNonexistentDataStream() { GetDataStreamAction.Request req = new GetDataStreamAction.Request(new String[] { dataStreamName }); IndexNotFoundException e = expectThrows( IndexNotFoundException.class, - () -> GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(), req) + () -> GetDataStreamsTransportAction.getDataStreams(cs, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), req) ); assertThat(e.getMessage(), containsString("no such index [" + dataStreamName + "]")); } diff --git a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/AbstractEnrichTestCase.java b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/AbstractEnrichTestCase.java index 88fdafa4de415..6894529dadbf8 100644 --- a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/AbstractEnrichTestCase.java +++ b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/AbstractEnrichTestCase.java @@ -10,6 +10,8 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.xpack.core.enrich.EnrichPolicy; @@ -31,7 +33,7 @@ protected AtomicReference saveEnrichPolicy(String name, EnrichPolicy if (policy != null) { createSourceIndices(policy); } - IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); + IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); CountDownLatch latch = new CountDownLatch(1); AtomicReference error = new AtomicReference<>(); EnrichStore.putPolicy(name, policy, clusterService, resolver, e -> { diff --git a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyExecutorTests.java b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyExecutorTests.java index fa9d43246775e..94088caae50f8 100644 --- a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyExecutorTests.java +++ b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyExecutorTests.java @@ -19,6 +19,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; @@ -141,7 +142,7 @@ public void testNonConcurrentPolicyExecution() throws InterruptedException { null, testTaskManager, testThreadPool, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), ESTestCase::randomNonNegativeLong ); @@ -198,7 +199,7 @@ public void testMaximumPolicyExecutionLimit() throws InterruptedException { null, testTaskManager, testThreadPool, - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), ESTestCase::randomNonNegativeLong ); diff --git a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyMaintenanceServiceTests.java b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyMaintenanceServiceTests.java index c26243fb9e40e..40c9c2e84d5ba 100644 --- a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyMaintenanceServiceTests.java +++ b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichPolicyMaintenanceServiceTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.mapper.MapperService; @@ -122,7 +123,7 @@ private EnrichPolicy randomPolicy() { } private void addPolicy(String policyName, EnrichPolicy policy) throws InterruptedException { - IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); + IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); createSourceIndices(client(), policy); doSyncronously( (clusterService, exceptionConsumer) -> EnrichStore.putPolicy(policyName, policy, clusterService, resolver, exceptionConsumer) diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java index b54923db6be6b..017ac06e615f5 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/AutodetectResultProcessorIT.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.reindex.ReindexPlugin; @@ -132,7 +133,8 @@ public void createComponents() throws Exception { Settings.Builder builder = Settings.builder() .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueSeconds(1)); AnomalyDetectionAuditor auditor = new AnomalyDetectionAuditor(client(), "test_node"); - jobResultsProvider = new JobResultsProvider(client(), builder.build(), new IndexNameExpressionResolver()); + jobResultsProvider = new JobResultsProvider(client(), builder.build(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); renormalizer = mock(Renormalizer.class); process = mock(AutodetectProcess.class); capturedUpdateModelSnapshotOnJobRequests = new ArrayList<>(); diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java index 85e125ed53262..7522e95dc286e 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/EstablishedMemUsageIT.java @@ -13,6 +13,7 @@ import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.ml.action.PutJobAction; @@ -59,7 +60,7 @@ public void createComponents() { OriginSettingClient originSettingClient = new OriginSettingClient(client(), ClientHelper.ML_ORIGIN); ResultsPersisterService resultsPersisterService = new ResultsPersisterService(originSettingClient, clusterService, settings); - jobResultsProvider = new JobResultsProvider(client(), settings, new IndexNameExpressionResolver()); + jobResultsProvider = new JobResultsProvider(client(), settings, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); jobResultsPersister = new JobResultsPersister( originSettingClient, resultsPersisterService, new AnomalyDetectionAuditor(client(), "test_node")); } diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java index 09e1f0c2e2c7c..1312bceaa91d3 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -107,7 +108,7 @@ public class JobResultsProviderIT extends MlSingleNodeTestCase { public void createComponents() throws Exception { Settings.Builder builder = Settings.builder() .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueSeconds(1)); - jobProvider = new JobResultsProvider(client(), builder.build(), new IndexNameExpressionResolver()); + jobProvider = new JobResultsProvider(client(), builder.build(), new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); ThreadPool tp = mock(ThreadPool.class); ClusterSettings clusterSettings = new ClusterSettings(builder.build(), new HashSet<>(Arrays.asList(InferenceProcessor.MAX_INFERENCE_PROCESSORS, diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobStorageDeletionTaskIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobStorageDeletionTaskIT.java index 744c46ad18359..ec1315ccc1fe6 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobStorageDeletionTaskIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/JobStorageDeletionTaskIT.java @@ -18,6 +18,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.threadpool.ThreadPool; @@ -74,7 +75,7 @@ public void createComponents() { ClusterService clusterService = new ClusterService(settings, clusterSettings, tp); OriginSettingClient originSettingClient = new OriginSettingClient(client(), ClientHelper.ML_ORIGIN); ResultsPersisterService resultsPersisterService = new ResultsPersisterService(originSettingClient, clusterService, settings); - jobResultsProvider = new JobResultsProvider(client(), settings, new IndexNameExpressionResolver()); + jobResultsProvider = new JobResultsProvider(client(), settings, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); jobResultsPersister = new JobResultsPersister( originSettingClient, resultsPersisterService, new AnomalyDetectionAuditor(client(), "test_node")); } diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlAutoUpdateServiceIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlAutoUpdateServiceIT.java index f0f3e66fe3f79..27ad678b21b7e 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlAutoUpdateServiceIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlAutoUpdateServiceIT.java @@ -16,6 +16,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.search.SearchModule; @@ -70,7 +71,7 @@ public void createComponents() throws Exception { public void testAutomaticModelUpdate() throws Exception { ensureGreen("_all"); - IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); client().prepareIndex(MlConfigIndex.indexName()) .setId(DatafeedConfig.documentId("farequote-datafeed-with-old-agg")) .setSource(AGG_WITH_OLD_DATE_HISTOGRAM_INTERVAL, XContentType.JSON) diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java index 2e8bbcc069cd2..97c40579d2d6b 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; @@ -75,7 +76,7 @@ public class MlConfigMigratorIT extends MlSingleNodeTestCase { - private final IndexNameExpressionResolver expressionResolver = new IndexNameExpressionResolver(); + private final IndexNameExpressionResolver expressionResolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); private ClusterService clusterService; @Before diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/UnusedStatsRemoverIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/UnusedStatsRemoverIT.java index 3ef0722d3f94c..2d2f106a575c0 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/UnusedStatsRemoverIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/UnusedStatsRemoverIT.java @@ -51,7 +51,8 @@ public class UnusedStatsRemoverIT extends BaseMlIntegTestCase { public void createComponents() { client = new OriginSettingClient(client(), ClientHelper.ML_ORIGIN); PlainActionFuture future = new PlainActionFuture<>(); - MlStatsIndex.createStatsIndexAndAliasIfNecessary(client(), clusterService().state(), new IndexNameExpressionResolver(), future); + MlStatsIndex.createStatsIndexAndAliasIfNecessary(client(), clusterService().state(), + new IndexNameExpressionResolver(client.threadPool().getThreadContext()), future); future.actionGet(); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java index a495d9a9d13e1..fbd98b8d9657d 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; @@ -92,7 +93,7 @@ public void testValidate_givenValidJob() { } public void testVerifyIndicesPrimaryShardsAreActive() { - final IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); + final IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); Metadata.Builder metadata = Metadata.builder(); RoutingTable.Builder routingTable = RoutingTable.builder(); addIndices(metadata, routingTable); @@ -106,7 +107,7 @@ public void testVerifyIndicesPrimaryShardsAreActive() { metadata = new Metadata.Builder(cs.metadata()); routingTable = new RoutingTable.Builder(cs.routingTable()); - IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); String indexToRemove = randomFrom(indexNameExpressionResolver.concreteIndexNames(cs, IndicesOptions.lenientExpandOpen(), TransportOpenJobAction.indicesOfInterest(".ml-anomalies-shared"))); if (randomBoolean()) { @@ -157,7 +158,7 @@ public void testGetAssignment_GivenJobThatRequiresMigration() { TransportOpenJobAction.OpenJobPersistentTasksExecutor executor = new TransportOpenJobAction.OpenJobPersistentTasksExecutor( Settings.EMPTY, clusterService, mock(AutodetectProcessManager.class), mock(MlMemoryTracker.class), mock(Client.class), - new IndexNameExpressionResolver()); + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); OpenJobAction.JobParams params = new OpenJobAction.JobParams("missing_job_field"); assertEquals(TransportOpenJobAction.AWAITING_MIGRATION, executor.getAssignment(params, mock(ClusterState.class))); @@ -183,7 +184,7 @@ public void testGetAssignment_GivenUnavailableIndicesWithLazyNode() { TransportOpenJobAction.OpenJobPersistentTasksExecutor executor = new TransportOpenJobAction.OpenJobPersistentTasksExecutor( settings, clusterService, mock(AutodetectProcessManager.class), mock(MlMemoryTracker.class), mock(Client.class), - new IndexNameExpressionResolver()); + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); OpenJobAction.JobParams params = new OpenJobAction.JobParams("unavailable_index_with_lazy_node"); params.setJob(mock(Job.class)); @@ -210,7 +211,7 @@ public void testGetAssignment_GivenLazyJobAndNoGlobalLazyNodes() { TransportOpenJobAction.OpenJobPersistentTasksExecutor executor = new TransportOpenJobAction.OpenJobPersistentTasksExecutor( settings, clusterService, mock(AutodetectProcessManager.class), mock(MlMemoryTracker.class), mock(Client.class), - new IndexNameExpressionResolver()); + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); Job job = mock(Job.class); when(job.allowLazyOpen()).thenReturn(true); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java index 79ef8422ee10b..c68f0d03f171f 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java @@ -18,6 +18,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; @@ -60,7 +61,8 @@ public void testVerifyIndicesPrimaryShardsAreActive() { ClusterState cs = csBuilder.build(); assertThat( - TransportStartDataFrameAnalyticsAction.verifyIndicesPrimaryShardsAreActive(cs, new IndexNameExpressionResolver(), indexName), + TransportStartDataFrameAnalyticsAction.verifyIndicesPrimaryShardsAreActive(cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), indexName), //G-> This is probably gonna break inside empty()); metadata = new Metadata.Builder(cs.metadata()); @@ -80,7 +82,7 @@ public void testVerifyIndicesPrimaryShardsAreActive() { csBuilder.routingTable(routingTable.build()); csBuilder.metadata(metadata); List result = TransportStartDataFrameAnalyticsAction.verifyIndicesPrimaryShardsAreActive(csBuilder.build(), - new IndexNameExpressionResolver(), indexName); + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), indexName); assertThat(result, contains(indexName)); } } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedConfigAutoUpdaterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedConfigAutoUpdaterTests.java index 4ac742231e3c9..a986ee693bbe6 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedConfigAutoUpdaterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedConfigAutoUpdaterTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; @@ -49,7 +50,7 @@ public class DatafeedConfigAutoUpdaterTests extends ESTestCase { private DatafeedConfigProvider provider; private List datafeeds = new ArrayList<>(); - private IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + private IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); @Before public void setup() { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedNodeSelectorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedNodeSelectorTests.java index cdddb03d095f5..d235a792dce53 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedNodeSelectorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedNodeSelectorTests.java @@ -25,7 +25,9 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.persistent.PersistentTasksCustomMetadata; @@ -63,7 +65,7 @@ public class DatafeedNodeSelectorTests extends ESTestCase { @Before public void init() { - resolver = new IndexNameExpressionResolver(); + resolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); nodes = DiscoveryNodes.builder() .add(new DiscoveryNode("node_name", "node_id", new TransportAddress(InetAddress.getLoopbackAddress(), 9300), Collections.emptyMap(), Collections.emptySet(), Version.CURRENT)) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/TrainedModelStatsServiceTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/TrainedModelStatsServiceTests.java index 972a90703d870..197507255483d 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/TrainedModelStatsServiceTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/TrainedModelStatsServiceTests.java @@ -20,6 +20,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; @@ -32,7 +33,7 @@ public class TrainedModelStatsServiceTests extends ESTestCase { public void testVerifyIndicesExistAndPrimaryShardsAreActive() { String aliasName = MlStatsIndex.writeAlias(); String concreteIndex = ".ml-stats-000001"; - IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); + IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); { Metadata.Builder metadata = Metadata.builder(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java index 49db60e884d61..d435848cbecc4 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java @@ -877,7 +877,7 @@ public void testCreateTermFieldsMapping() throws IOException { } private JobResultsProvider createProvider(Client client) { - return new JobResultsProvider(client, Settings.EMPTY, new IndexNameExpressionResolver()); + return new JobResultsProvider(client, Settings.EMPTY, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); } private static SearchResponse createSearchResponse(List> source) throws IOException { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java index fcc815e9f5af4..b76ec13853c06 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManagerTests.java @@ -713,7 +713,7 @@ private AutodetectProcessManager createManager(Settings settings) { return new AutodetectProcessManager(settings, client, threadPool, new NamedXContentRegistry(Collections.emptyList()), auditor, clusterService, jobManager, jobResultsProvider, jobResultsPersister, jobDataCountsPersister, annotationPersister, autodetectFactory, normalizerFactory, nativeStorageProvider, - new IndexNameExpressionResolver()); + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); } private AutodetectProcessManager createSpyManagerAndCallProcessData(String jobId) { AutodetectProcessManager manager = createSpyManager(); diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsCollectorTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsCollectorTests.java index cb7ef41a36411..3c43158f8b754 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsCollectorTests.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsCollectorTests.java @@ -19,6 +19,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.license.License; @@ -68,7 +69,7 @@ public void setUp() throws Exception { public void testShouldCollectReturnsFalseIfNotMaster() { final ClusterStatsCollector collector = new ClusterStatsCollector(Settings.EMPTY, clusterService, licenseState, client, licenseService, - new IndexNameExpressionResolver()); + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); assertThat(collector.shouldCollect(false), is(false)); } @@ -76,7 +77,7 @@ public void testShouldCollectReturnsFalseIfNotMaster() { public void testShouldCollectReturnsTrue() { final ClusterStatsCollector collector = new ClusterStatsCollector(Settings.EMPTY, clusterService, licenseState, client, licenseService, - new IndexNameExpressionResolver()); + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); assertThat(collector.shouldCollect(true), is(true)); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/DateMathExpressionIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/DateMathExpressionIntegTests.java index 5f3df136d1727..4158c56546d89 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/DateMathExpressionIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/DateMathExpressionIntegTests.java @@ -18,6 +18,8 @@ import org.elasticsearch.client.Requests; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.settings.SecureString; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.test.SecurityIntegTestCase; @@ -59,7 +61,8 @@ protected String configRoles() { public void testDateMathExpressionsCanBeAuthorized() throws Exception { final String expression = ""; - final String expectedIndexName = new IndexNameExpressionResolver().resolveDateMathExpression(expression); + final String expectedIndexName = + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)).resolveDateMathExpression(expression); final boolean refeshOnOperation = randomBoolean(); Client client = client().filterWithHeader(Collections.singletonMap("Authorization", basicAuthHeaderValue("user1", USERS_PASSWD))); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java index c4340ed707b25..a99380523b91e 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java @@ -130,7 +130,7 @@ protected SSLService getSslService() { when(client.threadPool()).thenReturn(threadPool); when(client.settings()).thenReturn(settings); return security.createComponents(client, threadPool, clusterService, mock(ResourceWatcherService.class), mock(ScriptService.class), - xContentRegistry(), env, new IndexNameExpressionResolver()); + xContentRegistry(), env, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); } private static T findComponent(Class type, Collection components) { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java index ac69ec5c50d02..fa59dcf5a8f13 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java @@ -258,7 +258,7 @@ public void setup() { roleMap.put(ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR.getName(), ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR); authorizationService = new AuthorizationService(settings, rolesStore, clusterService, auditTrailService, new DefaultAuthenticationFailureHandler(Collections.emptyMap()), threadPool, new AnonymousUser(settings), - null, Collections.emptySet(), licenseState, new IndexNameExpressionResolver()); + null, Collections.emptySet(), licenseState, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); } private void authorize(Authentication authentication, String action, TransportRequest request) { @@ -767,7 +767,7 @@ public void testDenialForAnonymousUser() throws IOException { final AnonymousUser anonymousUser = new AnonymousUser(settings); authorizationService = new AuthorizationService(settings, rolesStore, clusterService, auditTrailService, new DefaultAuthenticationFailureHandler(Collections.emptyMap()), threadPool, anonymousUser, null, Collections.emptySet(), - new XPackLicenseState(settings, () -> 0), new IndexNameExpressionResolver()); + new XPackLicenseState(settings, () -> 0), new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); RoleDescriptor role = new RoleDescriptor("a_all", null, new IndicesPrivileges[] { IndicesPrivileges.builder().indices("a").privileges("all").build() }, null); @@ -795,7 +795,8 @@ public void testDenialForAnonymousUserAuthorizationExceptionDisabled() throws IO final Authentication authentication = createAuthentication(new AnonymousUser(settings)); authorizationService = new AuthorizationService(settings, rolesStore, clusterService, auditTrailService, new DefaultAuthenticationFailureHandler(Collections.emptyMap()), threadPool, new AnonymousUser(settings), null, - Collections.emptySet(), new XPackLicenseState(settings, () -> 0), new IndexNameExpressionResolver()); + Collections.emptySet(), new XPackLicenseState(settings, () -> 0), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); RoleDescriptor role = new RoleDescriptor("a_all", null, new IndicesPrivileges[]{IndicesPrivileges.builder().indices("a").privileges("all").build()}, null); @@ -1539,7 +1540,8 @@ public void getUserPrivileges(Authentication authentication, AuthorizationInfo a when(licenseState.checkFeature(Feature.SECURITY_AUTHORIZATION_ENGINE)).thenReturn(true); authorizationService = new AuthorizationService(Settings.EMPTY, rolesStore, clusterService, auditTrailService, new DefaultAuthenticationFailureHandler(Collections.emptyMap()), threadPool, - new AnonymousUser(Settings.EMPTY), engine, Collections.emptySet(), licenseState, new IndexNameExpressionResolver()); + new AnonymousUser(Settings.EMPTY), engine, Collections.emptySet(), licenseState, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); Authentication authentication; try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { authentication = createAuthentication(new User("test user", "a_all")); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java index 16e5505a286e8..9c3d7b036c3e6 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java @@ -45,6 +45,7 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.protocol.xpack.graph.GraphExploreRequest; @@ -121,7 +122,7 @@ public void setup() { .put("cluster.remote.other_remote.seeds", "127.0.0.1:" + randomIntBetween(9351, 9399)) .build(); - indexNameExpressionResolver = new IndexNameExpressionResolver(); + indexNameExpressionResolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); final boolean withAlias = randomBoolean(); final String securityIndexName = SECURITY_MAIN_ALIAS + (withAlias ? "-" + randomAlphaOfLength(5) : ""); @@ -266,7 +267,8 @@ public void setup() { ClusterService clusterService = mock(ClusterService.class); when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); - defaultIndicesResolver = new IndicesAndAliasesResolver(settings, clusterService, new IndexNameExpressionResolver()); + defaultIndicesResolver = + new IndicesAndAliasesResolver(settings, clusterService, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); } public void testDashIndicesAreAllowedInShardLevelRequests() { diff --git a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformPersistentTasksExecutorTests.java b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformPersistentTasksExecutorTests.java index d6f622b61823c..ce7d513cc1aaa 100644 --- a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformPersistentTasksExecutorTests.java +++ b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformPersistentTasksExecutorTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.persistent.PersistentTasksCustomMetadata; @@ -178,7 +179,9 @@ public void testVerifyIndicesPrimaryShardsAreActive() { csBuilder.metadata(metadata); ClusterState cs = csBuilder.build(); - assertEquals(0, TransformPersistentTasksExecutor.verifyIndicesPrimaryShardsAreActive(cs, new IndexNameExpressionResolver()).size()); + assertEquals(0, + TransformPersistentTasksExecutor.verifyIndicesPrimaryShardsAreActive(cs, + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))).size()); metadata = new Metadata.Builder(cs.metadata()); routingTable = new RoutingTable.Builder(cs.routingTable()); @@ -204,7 +207,7 @@ public void testVerifyIndicesPrimaryShardsAreActive() { csBuilder.metadata(metadata); List result = TransformPersistentTasksExecutor.verifyIndicesPrimaryShardsAreActive( csBuilder.build(), - new IndexNameExpressionResolver() + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)) ); assertEquals(1, result.size()); assertEquals(indexToRemove, result.get(0)); @@ -391,7 +394,7 @@ public TransformPersistentTasksExecutor buildTaskExecutor() { mock(ThreadPool.class), clusterService, Settings.EMPTY, - new IndexNameExpressionResolver() + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)) ); } } diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherPluginTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherPluginTests.java index 2ea9a5be843bc..cf078e9e1b092 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherPluginTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherPluginTests.java @@ -7,6 +7,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexSettings; @@ -76,7 +77,7 @@ public void testWatcherDisabledTests() throws Exception { AnalysisRegistry registry = new AnalysisRegistry(TestEnvironment.newEnvironment(settings), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap(), emptyMap()); IndexModule indexModule = new IndexModule(indexSettings, registry, new InternalEngineFactory(), Collections.emptyMap(), - () -> true, new IndexNameExpressionResolver(), Collections.emptyMap()); + () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), Collections.emptyMap()); // this will trip an assertion if the watcher indexing operation listener is null (which it is) but we try to add it watcher.onIndexModule(indexModule); From 4712c4781157740b2d1579a93193fb3f9e0a7969 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 30 Jul 2020 11:15:16 -0600 Subject: [PATCH 002/161] Properly shut down components in test --- .../cluster/ClusterModuleTests.java | 20 ++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java b/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java index e122d5bb4c4ba..12a291f107116 100644 --- a/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java @@ -65,9 +65,23 @@ public class ClusterModuleTests extends ModuleTestCase { private ClusterInfoService clusterInfoService = EmptyClusterInfoService.INSTANCE; - private ClusterService clusterService = new ClusterService(Settings.EMPTY, - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null); - private ThreadPool threadPool = new TestThreadPool(this.getClass().getSimpleName() + "ThreadPool"); + private ClusterService clusterService; + private ThreadPool threadPool; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool(this.getClass().getSimpleName() + "ThreadPool"); + clusterService = new ClusterService(Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + clusterService.close(); + } static class FakeAllocationDecider extends AllocationDecider { protected FakeAllocationDecider() { From 6793e2e99a551bfbcc65aba22f4a7af9908e6776 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 30 Jul 2020 13:48:58 -0600 Subject: [PATCH 003/161] Version guard system index flag on cleanup --- .../java/org/elasticsearch/test/rest/ESRestTestCase.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index d7c0adf2540c5..28c8f9be23701 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -642,10 +642,13 @@ private void wipeCluster() throws Exception { protected static void wipeAllIndices() throws IOException { boolean includeHidden = minimumNodeVersion().onOrAfter(Version.V_7_7_0); + boolean includeSystem = minimumNodeVersion().onOrAfter(Version.V_8_0_0); try { final Request deleteRequest = new Request("DELETE", "*"); deleteRequest.addParameter("expand_wildcards", "open,closed" + (includeHidden ? ",hidden" : "")); - deleteRequest.addParameter("allow_system_index_access", "true"); + if (includeSystem) { + deleteRequest.addParameter("allow_system_index_access", "true"); + } final Response response = adminClient().performRequest(deleteRequest); try (InputStream is = response.getEntity().getContent()) { assertTrue((boolean) XContentHelper.convertToMap(XContentType.JSON.xContent(), is, true).get("acknowledged")); From 98cad266bbae56ab953e8e4b5eecb78ce594aa0b Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 3 Aug 2020 10:10:23 -0600 Subject: [PATCH 004/161] Fix guard in REST layer --- .../main/java/org/elasticsearch/rest/BaseRestHandler.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index fdf474bcccb4b..8bfa1cba984ac 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -77,8 +77,9 @@ public final long getUsageCount() { @Override public final void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { final String allow_system_index_access = request.param("allow_system_index_access"); - if (allow_system_index_access != null - && (allow_system_index_access.isEmpty() || Boolean.parseBoolean(allow_system_index_access) == false)) { + if (allow_system_index_access == null + || allow_system_index_access.isEmpty() + || Boolean.parseBoolean(allow_system_index_access) == false) { client.threadPool().getThreadContext().putHeader("_from_rest", "true"); } // prepare the request for execution; has the side effect of touching the request parameters From 3fe10113b75be52a5ef3dd9508798f3c2c065530 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 4 Aug 2020 15:33:33 -0600 Subject: [PATCH 005/161] Set origin from Enrich --- .../enrich/action/TransportExecuteEnrichPolicyAction.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportExecuteEnrichPolicyAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportExecuteEnrichPolicyAction.java index d567290dc906d..df3551fb4bfbf 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportExecuteEnrichPolicyAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportExecuteEnrichPolicyAction.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.Client; +import org.elasticsearch.client.OriginSettingClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; @@ -29,6 +30,8 @@ import java.io.IOException; +import static org.elasticsearch.xpack.core.ClientHelper.ENRICH_ORIGIN; + public class TransportExecuteEnrichPolicyAction extends TransportMasterNodeAction< ExecuteEnrichPolicyAction.Request, ExecuteEnrichPolicyAction.Response> { @@ -58,7 +61,7 @@ public TransportExecuteEnrichPolicyAction( this.executor = new EnrichPolicyExecutor( settings, clusterService, - client, + new OriginSettingClient(client, ENRICH_ORIGIN), transportService.getTaskManager(), threadPool, indexNameExpressionResolver, From 1c67a6fb54f5c514ceee101ea54444ef36008082 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 4 Aug 2020 15:47:05 -0600 Subject: [PATCH 006/161] Set origin from Enrich in the processor --- .../elasticsearch/xpack/enrich/AbstractEnrichProcessor.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/AbstractEnrichProcessor.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/AbstractEnrichProcessor.java index fc4d0691b39a3..525770d1fbd97 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/AbstractEnrichProcessor.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/AbstractEnrichProcessor.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Client; +import org.elasticsearch.client.OriginSettingClient; import org.elasticsearch.cluster.routing.Preference; import org.elasticsearch.index.query.ConstantScoreQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; @@ -25,6 +26,8 @@ import java.util.Map; import java.util.function.BiConsumer; +import static org.elasticsearch.xpack.core.ClientHelper.ENRICH_ORIGIN; + public abstract class AbstractEnrichProcessor extends AbstractProcessor { private final String policyName; @@ -188,8 +191,9 @@ int getMaxMatches() { } private static BiConsumer> createSearchRunner(Client client) { + Client originClient = new OriginSettingClient(client, ENRICH_ORIGIN); return (req, handler) -> { - client.execute( + originClient.execute( EnrichCoordinatorProxyAction.INSTANCE, req, ActionListener.wrap(resp -> { handler.accept(resp, null); }, e -> { handler.accept(null, e); }) From 57b8c036f0d7d3ae57310994a85f25ad999131de Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 4 Aug 2020 15:55:25 -0600 Subject: [PATCH 007/161] Fix NPEs in Enrich tests --- .../enrich/EnrichProcessorFactoryTests.java | 122 +++++++++--------- 1 file changed, 64 insertions(+), 58 deletions(-) diff --git a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichProcessorFactoryTests.java b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichProcessorFactoryTests.java index c2f31b063dee7..2c0321b2e2e57 100644 --- a/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichProcessorFactoryTests.java +++ b/x-pack/plugin/enrich/src/test/java/org/elasticsearch/xpack/enrich/EnrichProcessorFactoryTests.java @@ -7,6 +7,7 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.Version; +import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; @@ -14,6 +15,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.script.ScriptService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpClient; import org.elasticsearch.xpack.core.enrich.EnrichPolicy; import org.junit.Before; @@ -41,51 +43,53 @@ public void initializeScriptService() { public void testCreateProcessorInstance() throws Exception { List enrichValues = List.of("globalRank", "tldRank", "tld"); EnrichPolicy policy = new EnrichPolicy(EnrichPolicy.MATCH_TYPE, null, List.of("source_index"), "my_key", enrichValues); - EnrichProcessorFactory factory = new EnrichProcessorFactory(null, scriptService); - factory.metadata = createMetadata("majestic", policy); - - Map config = new HashMap<>(); - config.put("policy_name", "majestic"); - config.put("field", "host"); - config.put("target_field", "entry"); - boolean keyIgnoreMissing = randomBoolean(); - if (keyIgnoreMissing || randomBoolean()) { - config.put("ignore_missing", keyIgnoreMissing); - } - - Boolean overrideEnabled = randomBoolean() ? null : randomBoolean(); - if (overrideEnabled != null) { - config.put("override", overrideEnabled); - } - - Integer maxMatches = null; - if (randomBoolean()) { - maxMatches = randomIntBetween(1, 128); - config.put("max_matches", maxMatches); - } - - int numRandomValues = randomIntBetween(1, 8); - List> randomValues = new ArrayList<>(numRandomValues); - for (int i = 0; i < numRandomValues; i++) { - randomValues.add(new Tuple<>(randomFrom(enrichValues), randomAlphaOfLength(4))); - } - - MatchProcessor result = (MatchProcessor) factory.create(Collections.emptyMap(), "_tag", null, config); - assertThat(result, notNullValue()); - assertThat(result.getPolicyName(), equalTo("majestic")); - assertThat(result.getField(), equalTo("host")); - assertThat(result.getTargetField(), equalTo("entry")); - assertThat(result.getMatchField(), equalTo("my_key")); - assertThat(result.isIgnoreMissing(), is(keyIgnoreMissing)); - if (overrideEnabled != null) { - assertThat(result.isOverrideEnabled(), is(overrideEnabled)); - } else { - assertThat(result.isOverrideEnabled(), is(true)); - } - if (maxMatches != null) { - assertThat(result.getMaxMatches(), equalTo(maxMatches)); - } else { - assertThat(result.getMaxMatches(), equalTo(1)); + try (Client client = new NoOpClient(this.getClass().getSimpleName() + "TestClient")) { + EnrichProcessorFactory factory = new EnrichProcessorFactory(client, scriptService); + factory.metadata = createMetadata("majestic", policy); + + Map config = new HashMap<>(); + config.put("policy_name", "majestic"); + config.put("field", "host"); + config.put("target_field", "entry"); + boolean keyIgnoreMissing = randomBoolean(); + if (keyIgnoreMissing || randomBoolean()) { + config.put("ignore_missing", keyIgnoreMissing); + } + + Boolean overrideEnabled = randomBoolean() ? null : randomBoolean(); + if (overrideEnabled != null) { + config.put("override", overrideEnabled); + } + + Integer maxMatches = null; + if (randomBoolean()) { + maxMatches = randomIntBetween(1, 128); + config.put("max_matches", maxMatches); + } + + int numRandomValues = randomIntBetween(1, 8); + List> randomValues = new ArrayList<>(numRandomValues); + for (int i = 0; i < numRandomValues; i++) { + randomValues.add(new Tuple<>(randomFrom(enrichValues), randomAlphaOfLength(4))); + } + + MatchProcessor result = (MatchProcessor) factory.create(Collections.emptyMap(), "_tag", null, config); + assertThat(result, notNullValue()); + assertThat(result.getPolicyName(), equalTo("majestic")); + assertThat(result.getField(), equalTo("host")); + assertThat(result.getTargetField(), equalTo("entry")); + assertThat(result.getMatchField(), equalTo("my_key")); + assertThat(result.isIgnoreMissing(), is(keyIgnoreMissing)); + if (overrideEnabled != null) { + assertThat(result.isOverrideEnabled(), is(overrideEnabled)); + } else { + assertThat(result.isOverrideEnabled(), is(true)); + } + if (maxMatches != null) { + assertThat(result.getMaxMatches(), equalTo(maxMatches)); + } else { + assertThat(result.getMaxMatches(), equalTo(1)); + } } } @@ -167,19 +171,21 @@ public void testUnsupportedPolicy() throws Exception { public void testCompactEnrichValuesFormat() throws Exception { List enrichValues = List.of("globalRank", "tldRank", "tld"); EnrichPolicy policy = new EnrichPolicy(EnrichPolicy.MATCH_TYPE, null, List.of("source_index"), "host", enrichValues); - EnrichProcessorFactory factory = new EnrichProcessorFactory(null, scriptService); - factory.metadata = createMetadata("majestic", policy); - - Map config = new HashMap<>(); - config.put("policy_name", "majestic"); - config.put("field", "host"); - config.put("target_field", "entry"); - - MatchProcessor result = (MatchProcessor) factory.create(Collections.emptyMap(), "_tag", null, config); - assertThat(result, notNullValue()); - assertThat(result.getPolicyName(), equalTo("majestic")); - assertThat(result.getField(), equalTo("host")); - assertThat(result.getTargetField(), equalTo("entry")); + try (Client client = new NoOpClient(this.getClass().getSimpleName() + "TestClient")) { + EnrichProcessorFactory factory = new EnrichProcessorFactory(client, scriptService); + factory.metadata = createMetadata("majestic", policy); + + Map config = new HashMap<>(); + config.put("policy_name", "majestic"); + config.put("field", "host"); + config.put("target_field", "entry"); + + MatchProcessor result = (MatchProcessor) factory.create(Collections.emptyMap(), "_tag", null, config); + assertThat(result, notNullValue()); + assertThat(result.getPolicyName(), equalTo("majestic")); + assertThat(result.getField(), equalTo("host")); + assertThat(result.getTargetField(), equalTo("entry")); + } } public void testNoTargetField() throws Exception { From fc85715b2d901f9bdbdb330ffc04dfc4e05c2a1d Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 4 Aug 2020 16:04:23 -0600 Subject: [PATCH 008/161] Ensure we never double-add the header (should be tests only?) --- .../java/org/elasticsearch/rest/BaseRestHandler.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 8bfa1cba984ac..33f2f7ad58385 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.rest.action.admin.cluster.RestNodesUsageAction; @@ -77,10 +78,12 @@ public final long getUsageCount() { @Override public final void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { final String allow_system_index_access = request.param("allow_system_index_access"); - if (allow_system_index_access == null - || allow_system_index_access.isEmpty() - || Boolean.parseBoolean(allow_system_index_access) == false) { - client.threadPool().getThreadContext().putHeader("_from_rest", "true"); + final ThreadContext threadContext = client.threadPool().getThreadContext(); + if (threadContext.getHeader("_from_rest") == null + && (allow_system_index_access == null + || allow_system_index_access.isEmpty() + || Boolean.parseBoolean(allow_system_index_access) == false)) { + threadContext.putHeader("_from_rest", "true"); } // prepare the request for execution; has the side effect of touching the request parameters final RestChannelConsumer action = prepareRequest(request, client); From d782a40032f0976189f0eb7484332010455de537 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 4 Aug 2020 16:12:04 -0600 Subject: [PATCH 009/161] Fix NPE in UsageServiceTests --- .../usage/UsageServiceTests.java | 31 +++--- .../test/client/NoOpNodeClient.java | 99 +++++++++++++++++++ 2 files changed, 116 insertions(+), 14 deletions(-) create mode 100644 test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java diff --git a/server/src/test/java/org/elasticsearch/usage/UsageServiceTests.java b/server/src/test/java/org/elasticsearch/usage/UsageServiceTests.java index 947a64b3d34e2..83b596b364ccc 100644 --- a/server/src/test/java/org/elasticsearch/usage/UsageServiceTests.java +++ b/server/src/test/java/org/elasticsearch/usage/UsageServiceTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.rest.RestRequest; import org.elasticsearch.search.aggregations.support.AggregationUsageService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.test.rest.FakeRestRequest; import java.util.Collections; @@ -104,20 +105,22 @@ public void testRestUsage() throws Exception { usageService.addRestHandler(handlerD); usageService.addRestHandler(handlerE); usageService.addRestHandler(handlerF); - handlerA.handleRequest(restRequest, null, null); - handlerB.handleRequest(restRequest, null, null); - handlerA.handleRequest(restRequest, null, null); - handlerA.handleRequest(restRequest, null, null); - handlerB.handleRequest(restRequest, null, null); - handlerC.handleRequest(restRequest, null, null); - handlerC.handleRequest(restRequest, null, null); - handlerD.handleRequest(restRequest, null, null); - handlerA.handleRequest(restRequest, null, null); - handlerB.handleRequest(restRequest, null, null); - handlerE.handleRequest(restRequest, null, null); - handlerF.handleRequest(restRequest, null, null); - handlerC.handleRequest(restRequest, null, null); - handlerD.handleRequest(restRequest, null, null); + try (NodeClient client = new NoOpNodeClient(this.getClass().getSimpleName() + "TestClient")) { + handlerA.handleRequest(restRequest, null, client); + handlerB.handleRequest(restRequest, null, client); + handlerA.handleRequest(restRequest, null, client); + handlerA.handleRequest(restRequest, null, client); + handlerB.handleRequest(restRequest, null, client); + handlerC.handleRequest(restRequest, null, client); + handlerC.handleRequest(restRequest, null, client); + handlerD.handleRequest(restRequest, null, client); + handlerA.handleRequest(restRequest, null, client); + handlerB.handleRequest(restRequest, null, client); + handlerE.handleRequest(restRequest, null, client); + handlerF.handleRequest(restRequest, null, client); + handlerC.handleRequest(restRequest, null, client); + handlerD.handleRequest(restRequest, null, client); + } Map restUsage = usageService.getRestUsageStats(); assertThat(restUsage, notNullValue()); assertThat(restUsage.size(), equalTo(6)); diff --git a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java new file mode 100644 index 0000000000000..6f5a31ddb5e42 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java @@ -0,0 +1,99 @@ +/* + * 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. + */ + +package org.elasticsearch.test.client; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.support.TransportAction; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskListener; +import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.RemoteClusterService; + +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +public class NoOpNodeClient extends NodeClient { + + /** + * Build with {@link ThreadPool}. This {@linkplain ThreadPool} is terminated on {@link #close()}. + */ + public NoOpNodeClient(ThreadPool threadPool) { + super(Settings.EMPTY, threadPool); + } + + /** + * Create a new {@link TestThreadPool} for this client. + */ + public NoOpNodeClient(String testName) { + super(Settings.EMPTY, new TestThreadPool(testName)); + } + + @Override + public + void doExecute(ActionType action, Request request, ActionListener listener) { + listener.onResponse(null); + } + + @Override + public void initialize(Map actions, TaskManager taskManager, Supplier localNodeId, RemoteClusterService remoteClusterService) { + throw new UnsupportedOperationException("cannot initialize " + this.getClass().getSimpleName()); + } + + @Override + public Task executeLocally(ActionType action, Request request, ActionListener listener) { + listener.onResponse(null); + return null; + } + + @Override + public Task executeLocally(ActionType action, Request request, TaskListener listener) { + listener.onResponse(null, null); + return null; + } + + @Override + public String getLocalNodeId() { + return null; + } + + @Override + public Client getRemoteClusterClient(String clusterAlias) { + return null; + } + + @Override + public void close() { + try { + ThreadPool.terminate(threadPool(), 10, TimeUnit.SECONDS); + } catch (Exception e) { + throw new ElasticsearchException(e.getMessage(), e); + } + } +} From b437ad6c74945ee61a0787dbe3724a0f6359fd13 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 4 Aug 2020 16:13:52 -0600 Subject: [PATCH 010/161] Fix NPE in RestAnalyzeActionTests --- .../rest/action/admin/indices/RestAnalyzeActionTests.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestAnalyzeActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestAnalyzeActionTests.java index 8c3c9ae03cb2b..52569047c7680 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestAnalyzeActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestAnalyzeActionTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.rest.action.admin.indices; import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; +import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; @@ -26,6 +27,7 @@ import org.elasticsearch.index.analysis.NameOrDefinition; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.test.rest.FakeRestRequest; import java.io.IOException; @@ -95,8 +97,10 @@ public void testParseXContentForAnalyzeRequestWithInvalidJsonThrowsException() { RestAnalyzeAction action = new RestAnalyzeAction(); RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) .withContent(new BytesArray("{invalid_json}"), XContentType.JSON).build(); - IOException e = expectThrows(IOException.class, () -> action.handleRequest(request, null, null)); - assertThat(e.getMessage(), containsString("expecting double-quote")); + try (NodeClient client = new NoOpNodeClient(this.getClass().getSimpleName())) { + IOException e = expectThrows(IOException.class, () -> action.handleRequest(request, null, client)); + assertThat(e.getMessage(), containsString("expecting double-quote")); + } } public void testParseXContentForAnalyzeRequestWithUnknownParamThrowsException() throws Exception { From d0c1a8fe845fcbf7198dcebc3da697a50b5d03a3 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 4 Aug 2020 16:24:12 -0600 Subject: [PATCH 011/161] Fix NPE in RestBulkActionTests --- .../action/document/RestBulkActionTests.java | 69 ++++++++++--------- 1 file changed, 38 insertions(+), 31 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java index fabd9e36051f8..a068ef5e754a0 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java @@ -20,7 +20,9 @@ package org.elasticsearch.rest.action.document; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.bytes.BytesArray; @@ -28,15 +30,14 @@ import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.test.rest.FakeRestRequest; -import org.hamcrest.CustomMatcher; -import org.mockito.Mockito; import java.util.HashMap; import java.util.Map; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.argThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; import static org.mockito.Mockito.mock; /** @@ -45,32 +46,38 @@ public class RestBulkActionTests extends ESTestCase { public void testBulkPipelineUpsert() throws Exception { - final NodeClient mockClient = mock(NodeClient.class); - final Map params = new HashMap<>(); - params.put("pipeline", "timestamps"); - new RestBulkAction(settings(Version.CURRENT).build()) - .handleRequest( - new FakeRestRequest.Builder( - xContentRegistry()).withPath("my_index/_bulk").withParams(params) - .withContent( - new BytesArray( - "{\"index\":{\"_id\":\"1\"}}\n" + - "{\"field1\":\"val1\"}\n" + - "{\"update\":{\"_id\":\"2\"}}\n" + - "{\"script\":{\"source\":\"ctx._source.counter++;\"},\"upsert\":{\"field1\":\"upserted_val\"}}\n" - ), - XContentType.JSON - ).withMethod(RestRequest.Method.POST).build(), - mock(RestChannel.class), mockClient - ); - Mockito.verify(mockClient) - .bulk(argThat(new CustomMatcher("Pipeline in upsert request") { - @Override - public boolean matches(final Object item) { - BulkRequest request = (BulkRequest) item; - UpdateRequest update = (UpdateRequest) request.requests().get(1); - return "timestamps".equals(update.upsertRequest().getPipeline()); - } - }), any()); + try (NodeClient verifyingClient = new VerifyingClient(this.getTestName())) { + final Map params = new HashMap<>(); + params.put("pipeline", "timestamps"); + new RestBulkAction(settings(Version.CURRENT).build()) + .handleRequest( + new FakeRestRequest.Builder( + xContentRegistry()).withPath("my_index/_bulk").withParams(params) + .withContent( + new BytesArray( + "{\"index\":{\"_id\":\"1\"}}\n" + + "{\"field1\":\"val1\"}\n" + + "{\"update\":{\"_id\":\"2\"}}\n" + + "{\"script\":{\"source\":\"ctx._source.counter++;\"},\"upsert\":{\"field1\":\"upserted_val\"}}\n" + ), + XContentType.JSON + ).withMethod(RestRequest.Method.POST).build(), + mock(RestChannel.class), verifyingClient + ); + + } + } + + private static class VerifyingClient extends NoOpNodeClient { + public VerifyingClient(String testName) { + super(testName); + } + + @Override + public void bulk(BulkRequest request, ActionListener listener) { + assertThat(request.requests(), hasSize(2)); + UpdateRequest updateRequest = (UpdateRequest) request.requests().get(1); + assertThat(updateRequest.upsertRequest().getPipeline(), equalTo("timestamps")); + } } } From ebaafcfb72bbbfcb9c268fb4fe15754c0dc85c48 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 4 Aug 2020 16:48:19 -0600 Subject: [PATCH 012/161] Fix NPE/mocking in `RestActionTestCase`s --- .../action/document/RestBulkActionTests.java | 2 +- .../action/document/RestIndexActionTests.java | 18 +++-- .../test/client/NoOpNodeClient.java | 9 ++- .../test/rest/RestActionTestCase.java | 74 +++++++++++++++++-- .../RestSubmitAsyncSearchActionTests.java | 55 ++++++++------ 5 files changed, 116 insertions(+), 42 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java index a068ef5e754a0..550d8075e27da 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java @@ -69,7 +69,7 @@ public void testBulkPipelineUpsert() throws Exception { } private static class VerifyingClient extends NoOpNodeClient { - public VerifyingClient(String testName) { + VerifyingClient(String testName) { super(testName); } diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestIndexActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestIndexActionTests.java index bb549b64c724f..cafe96a0c4d55 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestIndexActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestIndexActionTests.java @@ -19,8 +19,8 @@ package org.elasticsearch.rest.action.document; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.cluster.ClusterName; @@ -36,13 +36,11 @@ import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.test.rest.RestActionTestCase; import org.junit.Before; -import org.mockito.ArgumentCaptor; import java.util.concurrent.atomic.AtomicReference; import static org.hamcrest.Matchers.equalTo; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.verify; +import static org.hamcrest.Matchers.instanceOf; public class RestIndexActionTests extends RestActionTestCase { @@ -76,6 +74,13 @@ public void testAutoIdDefaultsToOptypeIndexForOlderVersions() { } private void checkAutoIdOpType(Version minClusterVersion, DocWriteRequest.OpType expectedOpType) { + SetOnce executeCalled = new SetOnce<>(); + verifyingClient.setExecuteVerifier((actionType, request) -> { + assertThat(request, instanceOf(IndexRequest.class)); + assertThat(((IndexRequest) request).opType(), equalTo(expectedOpType)); + executeCalled.set(true); + return null; + }); RestRequest autoIdRequest = new FakeRestRequest.Builder(xContentRegistry()) .withMethod(RestRequest.Method.POST) .withPath("/some_index/_doc") @@ -86,9 +91,6 @@ private void checkAutoIdOpType(Version minClusterVersion, DocWriteRequest.OpType .add(new DiscoveryNode("test", buildNewFakeTransportAddress(), minClusterVersion)) .build()).build()); dispatchRequest(autoIdRequest); - ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(IndexRequest.class); - verify(nodeClient).index(argumentCaptor.capture(), any(ActionListener.class)); - IndexRequest indexRequest = argumentCaptor.getValue(); - assertEquals(expectedOpType, indexRequest.opType()); + assertThat(executeCalled.get(), equalTo(true)); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java index 6f5a31ddb5e42..b75766fb3142f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java @@ -62,18 +62,21 @@ void doExecute(ActionType action, Request request, ActionListener actions, TaskManager taskManager, Supplier localNodeId, RemoteClusterService remoteClusterService) { + public void initialize(Map actions, TaskManager taskManager, Supplier localNodeId, + RemoteClusterService remoteClusterService) { throw new UnsupportedOperationException("cannot initialize " + this.getClass().getSimpleName()); } @Override - public Task executeLocally(ActionType action, Request request, ActionListener listener) { + public + Task executeLocally(ActionType action, Request request, ActionListener listener) { listener.onResponse(null); return null; } @Override - public Task executeLocally(ActionType action, Request request, TaskListener listener) { + public + Task executeLocally(ActionType action, Request request, TaskListener listener) { listener.onResponse(null, null); return null; } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/RestActionTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/RestActionTestCase.java index a5d932a3d1a3d..8adaccfe07a56 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/RestActionTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/RestActionTestCase.java @@ -19,19 +19,26 @@ package org.elasticsearch.test.rest; -import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionType; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskListener; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.usage.UsageService; +import org.junit.After; import org.junit.Before; import java.util.Collections; - -import static org.mockito.Mockito.mock; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; /** * A common base class for Rest*ActionTests. Provides access to a {@link RestController} @@ -39,17 +46,22 @@ */ public abstract class RestActionTestCase extends ESTestCase { private RestController controller; - protected NodeClient nodeClient; + protected VerifyingClient verifyingClient; @Before public void setUpController() { - nodeClient = mock(NodeClient.class); + verifyingClient = new VerifyingClient(this.getTestName()); controller = new RestController(Collections.emptySet(), null, - nodeClient, + verifyingClient, new NoneCircuitBreakerService(), new UsageService()); } + @After + public void tearDownController() { + verifyingClient.close(); + } + /** * A test {@link RestController}. This controller can be used to register and delegate * to handlers, but uses a mock client and cannot carry out the full request. @@ -66,4 +78,54 @@ protected void dispatchRequest(RestRequest request) { ThreadContext threadContext = new ThreadContext(Settings.EMPTY); controller.dispatchRequest(request, channel, threadContext); } + + public static class VerifyingClient extends NoOpNodeClient { + AtomicReference executeVerifier = new AtomicReference<>(); + AtomicReference executeLocallyVerifier = new AtomicReference<>(); + + public VerifyingClient(String testName) { + super(testName); + reset(); + } + + public void reset() { + executeVerifier.set((arg1, arg2) -> { + throw new AssertionError(); + }); + executeLocallyVerifier.set((arg1, arg2) -> { + throw new AssertionError(); + }); + } + + public + void setExecuteVerifier(BiFunction, Request, Void> verifier) { + executeVerifier.set(verifier); + } + + @Override + public + void doExecute(ActionType action, Request request, ActionListener listener) { + listener.onResponse((Response) executeVerifier.get().apply(action, request)); + } + + public + void setExecuteLocallyVerifier(BiFunction, Request, Void> verifier) { + executeLocallyVerifier.set(verifier); + } + + @Override + public + Task executeLocally(ActionType action, Request request, ActionListener listener) { + listener.onResponse((Response) executeLocallyVerifier.get().apply(action, request)); + return null; + } + + @Override + public + Task executeLocally(ActionType action, Request request, TaskListener listener) { + listener.onResponse(null, (Response) executeLocallyVerifier.get().apply(action, request)); + return null; + } + + } } diff --git a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/RestSubmitAsyncSearchActionTests.java b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/RestSubmitAsyncSearchActionTests.java index 045dd153f9853..54e297cf25fba 100644 --- a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/RestSubmitAsyncSearchActionTests.java +++ b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/RestSubmitAsyncSearchActionTests.java @@ -5,8 +5,7 @@ */ package org.elasticsearch.xpack.search; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionType; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; @@ -15,16 +14,14 @@ import org.elasticsearch.test.rest.RestActionTestCase; import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest; import org.junit.Before; -import org.mockito.ArgumentCaptor; import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.function.Function; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.reset; -import static org.mockito.Mockito.verify; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; public class RestSubmitAsyncSearchActionTests extends RestActionTestCase { @@ -42,26 +39,31 @@ public void setUpAction() { */ @SuppressWarnings("unchecked") public void testRequestParameterDefaults() throws IOException { + SetOnce executeCalled = new SetOnce<>(); + verifyingClient.setExecuteLocallyVerifier((actionType, request) -> { + assertThat(request, instanceOf(SubmitAsyncSearchRequest.class)); + SubmitAsyncSearchRequest submitRequest = (SubmitAsyncSearchRequest) request; + assertThat(submitRequest.getWaitForCompletionTimeout(), equalTo(TimeValue.timeValueSeconds(1))); + assertThat(submitRequest.isKeepOnCompletion(), equalTo(false)); + assertThat(submitRequest.getKeepAlive(), equalTo(TimeValue.timeValueDays(5))); + // check parameters we implicitly set in the SubmitAsyncSearchRequest ctor + assertThat(submitRequest.getSearchRequest().isCcsMinimizeRoundtrips(), equalTo(false)); + assertThat(submitRequest.getSearchRequest().getBatchedReduceSize(), equalTo(5)); + assertThat(submitRequest.getSearchRequest().requestCache(), equalTo(true)); + assertThat(submitRequest.getSearchRequest().getPreFilterShardSize().intValue(), equalTo(1)); + executeCalled.set(true); + return null; + }); RestRequest submitAsyncRestRequest = new FakeRestRequest.Builder(xContentRegistry()) .withMethod(RestRequest.Method.POST) .withPath("/test_index/_async_search") .withContent(new BytesArray("{}"), XContentType.JSON) .build(); dispatchRequest(submitAsyncRestRequest); - ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(SubmitAsyncSearchRequest.class); - verify(nodeClient).executeLocally(any(ActionType.class), argumentCaptor.capture(), any(ActionListener.class)); - SubmitAsyncSearchRequest submitRequest = argumentCaptor.getValue(); - assertEquals(TimeValue.timeValueSeconds(1), submitRequest.getWaitForCompletionTimeout()); - assertFalse(submitRequest.isKeepOnCompletion()); - assertEquals(TimeValue.timeValueDays(5), submitRequest.getKeepAlive()); - // check parameters we implicitly set in the SubmitAsyncSearchRequest ctor - assertFalse(submitRequest.getSearchRequest().isCcsMinimizeRoundtrips()); - assertEquals(5, submitRequest.getSearchRequest().getBatchedReduceSize()); - assertEquals(true, submitRequest.getSearchRequest().requestCache()); - assertEquals(1, submitRequest.getSearchRequest().getPreFilterShardSize().intValue()); + assertThat(executeCalled.get(), equalTo(true)); } - public void testParameters() throws IOException { + public void testParameters() throws Exception { String tvString = randomTimeValue(1, 100); doTestParameter("keep_alive", tvString, TimeValue.parseTimeValue(tvString, ""), SubmitAsyncSearchRequest::getKeepAlive); doTestParameter("wait_for_completion_timeout", tvString, TimeValue.parseTimeValue(tvString, ""), @@ -79,18 +81,23 @@ public void testParameters() throws IOException { @SuppressWarnings("unchecked") private void doTestParameter(String paramName, String paramValue, T expectedValue, - Function valueAccessor) { + Function valueAccessor) throws Exception { + SetOnce executeCalled = new SetOnce<>(); + verifyingClient.setExecuteLocallyVerifier((actionType, request) -> { + assertThat(request, instanceOf(SubmitAsyncSearchRequest.class)); + assertThat(valueAccessor.apply((SubmitAsyncSearchRequest) request), equalTo(expectedValue)); + executeCalled.set(true); + return null; + }); Map params = new HashMap<>(); params.put(paramName, paramValue); RestRequest submitAsyncRestRequest = new FakeRestRequest.Builder(xContentRegistry()).withMethod(RestRequest.Method.POST) .withPath("/test_index/_async_search") .withParams(params) .withContent(new BytesArray("{}"), XContentType.JSON).build(); - ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(SubmitAsyncSearchRequest.class); + dispatchRequest(submitAsyncRestRequest); - verify(nodeClient).executeLocally(any(ActionType.class), argumentCaptor.capture(), any(ActionListener.class)); - SubmitAsyncSearchRequest submitRequest = argumentCaptor.getValue(); - assertEquals(expectedValue, valueAccessor.apply(submitRequest)); - reset(nodeClient); + assertThat(executeCalled.get(), equalTo(true)); + verifyingClient.reset(); } } From 05f8fa1e34cb9b29ce4ccd758add8c3b71c7273e Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 13:51:12 -0600 Subject: [PATCH 013/161] Add `allow_system_index_access` flag to search spec --- .../src/main/resources/rest-api-spec/api/search.json | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index 03ca18705817e..67c86c0ae23c3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -225,6 +225,10 @@ "type":"boolean", "description":"Indicates whether hits.total should be rendered as an integer or an object in the rest search response", "default":false + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ From 606d9c93186810b02ff12bd5be594589d6a97ea8 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 13:51:26 -0600 Subject: [PATCH 014/161] Add `allow_system_index_access` as necessary to Watcher tests --- .../rest-api-spec/test/watcher/ack_watch/10_basic.yml | 1 + .../rest-api-spec/test/watcher/activate_watch/10_basic.yml | 2 ++ .../rest-api-spec/test/watcher/get_watch/10_basic.yml | 1 + .../watcher/put_watch/80_put_get_watch_with_passwords.yml | 1 + .../smoketest/MonitoringWithWatcherRestIT.java | 6 ++++-- 5 files changed, 9 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/ack_watch/10_basic.yml b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/ack_watch/10_basic.yml index ed35d17984679..9fa8c627394d3 100644 --- a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/ack_watch/10_basic.yml +++ b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/ack_watch/10_basic.yml @@ -46,6 +46,7 @@ - do: search: rest_total_hits_as_int: true + allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 1 } diff --git a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/activate_watch/10_basic.yml b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/activate_watch/10_basic.yml index 5f09e7ef1847a..ecf873857801e 100644 --- a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/activate_watch/10_basic.yml +++ b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/activate_watch/10_basic.yml @@ -50,6 +50,7 @@ - do: search: rest_total_hits_as_int: true + allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 1 } @@ -71,6 +72,7 @@ - do: search: rest_total_hits_as_int: true + allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 1 } diff --git a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/get_watch/10_basic.yml b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/get_watch/10_basic.yml index 09b2230f04c60..626adeebb5fac 100644 --- a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/get_watch/10_basic.yml +++ b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/get_watch/10_basic.yml @@ -49,6 +49,7 @@ teardown: - do: search: rest_total_hits_as_int: true + allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 1 } diff --git a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml index 02191f0b680a9..a1cd025c72df3 100644 --- a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml +++ b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml @@ -262,6 +262,7 @@ setup: - do: search: rest_total_hits_as_int: true + allow_system_index_access: true index: .watches body: > { diff --git a/x-pack/plugin/watcher/qa/with-monitoring/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java b/x-pack/plugin/watcher/qa/with-monitoring/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java index cac2268f20119..34a29adac02cb 100644 --- a/x-pack/plugin/watcher/qa/with-monitoring/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java +++ b/x-pack/plugin/watcher/qa/with-monitoring/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java @@ -86,8 +86,10 @@ private void assertMonitoringWatchHasBeenOverWritten(String watchId) throws Exce private void assertTotalWatchCount(int expectedWatches) throws Exception { assertBusy(() -> { - assertOK(client().performRequest(new Request("POST", "/.watches/_refresh"))); - ObjectPath path = ObjectPath.createFromResponse(client().performRequest(new Request("POST", "/.watches/_count"))); + refreshAllIndices(); + final Request countRequest = new Request("POST", "/.watches/_count"); + countRequest.addParameter("allow_system_index_access", "true"); + ObjectPath path = ObjectPath.createFromResponse(client().performRequest(countRequest)); int count = path.evaluate("count"); assertThat(count, is(expectedWatches)); }); From d74939b0bbd1a9a90f45b00e933b9912456fec76 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 14:12:16 -0600 Subject: [PATCH 015/161] Add `allow_system_index_access` flag to count, index, index.delete specs --- rest-api-spec/src/main/resources/rest-api-spec/api/count.json | 4 ++++ rest-api-spec/src/main/resources/rest-api-spec/api/index.json | 4 ++++ .../src/main/resources/rest-api-spec/api/indices.delete.json | 4 ++++ 3 files changed, 12 insertions(+) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json index 93a450afff88a..cee111131e1e3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json @@ -98,6 +98,10 @@ "terminate_after":{ "type":"number", "description":"The maximum count for each shard, upon reaching which the query execution will terminate early" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json index b4865403331b0..3da3a66ab8a1b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json @@ -96,6 +96,10 @@ "require_alias": { "type": "boolean", "description": "When true, requires destination to be an alias. Default is false" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json index 53fdf44bb36a1..243953b364204 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json @@ -49,6 +49,10 @@ ], "default":"open", "description":"Whether wildcard expressions should get expanded to open or closed indices (default: open)" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } From 212224098ed188c1298a269e565e0f308603a6c0 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 14:12:38 -0600 Subject: [PATCH 016/161] Add `allow_system_index_access` to ML REST tests as necessary --- .../src/test/resources/rest-api-spec/test/ml/calendar_crud.yml | 2 ++ .../src/test/resources/rest-api-spec/test/ml/filter_crud.yml | 2 ++ .../resources/rest-api-spec/test/ml/inference_stats_crud.yml | 3 +++ 3 files changed, 7 insertions(+) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/calendar_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/calendar_crud.yml index b4b5c9ad9da7a..ae3f051cdb273 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/calendar_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/calendar_crud.yml @@ -422,6 +422,7 @@ # Check the event from calendar 1 is deleted - do: count: + allow_system_index_access: true index: .ml-meta body: query: @@ -433,6 +434,7 @@ - do: count: + allow_system_index_access: true index: .ml-meta body: query: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml index aa44f928ae508..63cbac7515b0c 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml @@ -12,6 +12,7 @@ setup: - "index [.ml-meta] matches multiple legacy templates [.ml-meta, global], composable templates will only match a single template" index: index: .ml-meta + allow_system_index_access: true id: filter_imposter-filter body: > { @@ -322,6 +323,7 @@ setup: - do: indices.delete: + allow_system_index_access: true index: ".ml-meta" - do: allowed_warnings: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml index 1b5aa8bbc4929..c73e1b456ed4f 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml @@ -9,6 +9,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: + allow_system_index_access: true id: trained_model_config-a-unused-regression-model1-0 index: .ml-inference-000002 body: > @@ -26,6 +27,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: + allow_system_index_access: true id: trained_model_config-a-unused-regression-model-0 index: .ml-inference-000002 body: > @@ -42,6 +44,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: + allow_system_index_access: true id: trained_model_config-a-used-regression-model-0 index: .ml-inference-000002 body: > From a618ec1b7ff153b082c243c0e374f7bea4bd1f33 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 15:01:56 -0600 Subject: [PATCH 017/161] Allow REST handlers to control whether they allow system index access --- .../elasticsearch/rest/BaseRestHandler.java | 20 ++++++++++++++----- .../admin/indices/RestRefreshAction.java | 5 +++++ 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 33f2f7ad58385..63f109ac3cb78 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -77,13 +77,15 @@ public final long getUsageCount() { @Override public final void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { - final String allow_system_index_access = request.param("allow_system_index_access"); - final ThreadContext threadContext = client.threadPool().getThreadContext(); - if (threadContext.getHeader("_from_rest") == null - && (allow_system_index_access == null + if (allowSystemIndexAccessByDefault() == false) { + final String allow_system_index_access = request.param("allow_system_index_access"); + final ThreadContext threadContext = client.threadPool().getThreadContext(); + if (threadContext.getHeader("_from_rest") == null + && (allow_system_index_access == null || allow_system_index_access.isEmpty() || Boolean.parseBoolean(allow_system_index_access) == false)) { - threadContext.putHeader("_from_rest", "true"); + threadContext.putHeader("_from_rest", "true"); + } } // prepare the request for execution; has the side effect of touching the request parameters final RestChannelConsumer action = prepareRequest(request, client); @@ -192,6 +194,14 @@ protected Set responseParams() { return Collections.emptySet(); } + /** + * Controls whether requests handled by this class are allowed to to access system indices by default. + * @return {@code true} if requests handled by this class should be allowed to access system indices. + */ + protected boolean allowSystemIndexAccessByDefault() { + return false; + } + public static class Wrapper extends BaseRestHandler { protected final BaseRestHandler delegate; diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java index 3d9cbd903bb70..ac4060d32c030 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java @@ -51,6 +51,11 @@ public String getName() { return "refresh_action"; } + @Override + protected boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { RefreshRequest refreshRequest = new RefreshRequest(Strings.splitStringByCommaToArray(request.param("index"))); From 842e3efba76b5aa63c73a4353b4b2f9c59f63380 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 17:54:26 -0600 Subject: [PATCH 018/161] Fix NPE/client mock in RestSearchScrollActionTests --- .../scroll/RestSearchScrollActionTests.java | 43 ++++++++++--------- 1 file changed, 22 insertions(+), 21 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/scroll/RestSearchScrollActionTests.java b/server/src/test/java/org/elasticsearch/search/scroll/RestSearchScrollActionTests.java index 6c58d32b8b8bb..986c72c1bbe9e 100644 --- a/server/src/test/java/org/elasticsearch/search/scroll/RestSearchScrollActionTests.java +++ b/server/src/test/java/org/elasticsearch/search/scroll/RestSearchScrollActionTests.java @@ -19,6 +19,9 @@ package org.elasticsearch.search.scroll; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.bytes.BytesArray; @@ -26,19 +29,14 @@ import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.search.RestSearchScrollAction; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.test.rest.FakeRestChannel; import org.elasticsearch.test.rest.FakeRestRequest; -import org.mockito.ArgumentCaptor; import java.util.HashMap; import java.util.Map; import static org.hamcrest.Matchers.equalTo; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; public class RestSearchScrollActionTests extends ESTestCase { @@ -51,23 +49,26 @@ public void testParseSearchScrollRequestWithInvalidJsonThrowsException() throws } public void testBodyParamsOverrideQueryStringParams() throws Exception { - NodeClient nodeClient = mock(NodeClient.class); - doNothing().when(nodeClient).searchScroll(any(), any()); - - RestSearchScrollAction action = new RestSearchScrollAction(); - Map params = new HashMap<>(); - params.put("scroll_id", "QUERY_STRING"); - params.put("scroll", "1000m"); - RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) + SetOnce scrollCalled = new SetOnce<>(); + try (NodeClient nodeClient = new NoOpNodeClient(this.getTestName()) { + @Override + public void searchScroll(SearchScrollRequest request, ActionListener listener) { + scrollCalled.set(true); + assertThat(request.scrollId(), equalTo("BODY")); + assertThat(request.scroll().keepAlive().getStringRep(), equalTo("1m")); + } + }) { + RestSearchScrollAction action = new RestSearchScrollAction(); + Map params = new HashMap<>(); + params.put("scroll_id", "QUERY_STRING"); + params.put("scroll", "1000m"); + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) .withParams(params) .withContent(new BytesArray("{\"scroll_id\":\"BODY\", \"scroll\":\"1m\"}"), XContentType.JSON).build(); - FakeRestChannel channel = new FakeRestChannel(request, false, 0); - action.handleRequest(request, channel, nodeClient); + FakeRestChannel channel = new FakeRestChannel(request, false, 0); + action.handleRequest(request, channel, nodeClient); - ArgumentCaptor argument = ArgumentCaptor.forClass(SearchScrollRequest.class); - verify(nodeClient).searchScroll(argument.capture(), anyObject()); - SearchScrollRequest searchScrollRequest = argument.getValue(); - assertEquals("BODY", searchScrollRequest.scrollId()); - assertEquals("1m", searchScrollRequest.scroll().keepAlive().getStringRep()); + assertThat(scrollCalled.get(), equalTo(true)); + } } } From d2acd60b674136da847c2e821d5e8c25b48dbc2b Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 17:56:54 -0600 Subject: [PATCH 019/161] Fix NPE/client mock in RestClearScrollActionTests --- .../scroll/RestClearScrollActionTests.java | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/scroll/RestClearScrollActionTests.java b/server/src/test/java/org/elasticsearch/search/scroll/RestClearScrollActionTests.java index 03e9a242d35c0..8618b6b8de873 100644 --- a/server/src/test/java/org/elasticsearch/search/scroll/RestClearScrollActionTests.java +++ b/server/src/test/java/org/elasticsearch/search/scroll/RestClearScrollActionTests.java @@ -19,26 +19,24 @@ package org.elasticsearch.search.scroll; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.search.ClearScrollRequest; +import org.elasticsearch.action.search.ClearScrollResponse; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.search.RestClearScrollAction; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.test.rest.FakeRestChannel; import org.elasticsearch.test.rest.FakeRestRequest; -import org.mockito.ArgumentCaptor; import java.util.Collections; -import java.util.List; import static org.hamcrest.Matchers.equalTo; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyObject; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; +import static org.hamcrest.Matchers.hasSize; public class RestClearScrollActionTests extends ESTestCase { @@ -51,21 +49,23 @@ public void testParseClearScrollRequestWithInvalidJsonThrowsException() throws E } public void testBodyParamsOverrideQueryStringParams() throws Exception { - NodeClient nodeClient = mock(NodeClient.class); - doNothing().when(nodeClient).searchScroll(any(), any()); - - RestClearScrollAction action = new RestClearScrollAction(); - RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) + SetOnce scrollCalled = new SetOnce<>(); + try (NodeClient nodeClient = new NoOpNodeClient(this.getTestName()) { + @Override + public void clearScroll(ClearScrollRequest request, ActionListener listener) { + scrollCalled.set(true); + assertThat(request.getScrollIds(), hasSize(1)); + assertThat(request.getScrollIds().get(0), equalTo("BODY")); + } + }) { + RestClearScrollAction action = new RestClearScrollAction(); + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) .withParams(Collections.singletonMap("scroll_id", "QUERY_STRING")) .withContent(new BytesArray("{\"scroll_id\": [\"BODY\"]}"), XContentType.JSON).build(); - FakeRestChannel channel = new FakeRestChannel(request, false, 0); - action.handleRequest(request, channel, nodeClient); + FakeRestChannel channel = new FakeRestChannel(request, false, 0); + action.handleRequest(request, channel, nodeClient); - ArgumentCaptor argument = ArgumentCaptor.forClass(ClearScrollRequest.class); - verify(nodeClient).clearScroll(argument.capture(), anyObject()); - ClearScrollRequest clearScrollRequest = argument.getValue(); - List scrollIds = clearScrollRequest.getScrollIds(); - assertEquals(1, scrollIds.size()); - assertEquals("BODY", scrollIds.get(0)); + assertThat(scrollCalled.get(), equalTo(true)); + } } } From 04eb2fa6483211a2b9870cc290dfe0ed2f9173a3 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 18:04:12 -0600 Subject: [PATCH 020/161] Add allow_system_index_access flag to Get API --- rest-api-spec/src/main/resources/rest-api-spec/api/get.json | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json index 36d08c0313bde..bb2d8212af0cf 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json @@ -70,6 +70,10 @@ "external_gte" ], "description":"Specific version type" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } From a7ea769a29e935b71176fb2a2190be22e5f251af Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 18:04:50 -0600 Subject: [PATCH 021/161] Add allow_system_index_access flag to Security tests as necessary --- .../src/test/resources/rest-api-spec/test/users/10_basic.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml index 9f992adde9670..e560b1d5204c2 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml @@ -123,6 +123,7 @@ teardown: - do: get: + allow_system_index_access: true index: .security id: user-bob - set: { _source.password: "hash" } From 05b80ab1923b4e1c3390575542a2634f3c86ba3a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 18:09:14 -0600 Subject: [PATCH 022/161] Add allow_system_index_access flag to upgrade tests as necessary --- .../resources/rest-api-spec/test/upgraded_cluster/10_basic.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml index 10fded326855c..67bd9de46e575 100644 --- a/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml +++ b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml @@ -91,6 +91,7 @@ - do: search: + allow_system_index_access: true rest_total_hits_as_int: true index: .tasks body: From ce93851d018be56416e61df54a7f4f5ac84ae6a8 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 18:12:22 -0600 Subject: [PATCH 023/161] Include system indices in cluster health by default --- .../rest/action/admin/cluster/RestClusterHealthAction.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java index e43dd4e7c9428..d72aec47eadca 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java @@ -52,6 +52,11 @@ public String getName() { return "cluster_health_action"; } + @Override + protected boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { final ClusterHealthRequest clusterHealthRequest = fromRequest(request); From fc88e4183f3740bbabe1a18a972a4b7fc5b640b4 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 18:15:31 -0600 Subject: [PATCH 024/161] Fix NPE/client mock in SecurityBaseRestHandlerTests --- .../action/SecurityBaseRestHandlerTests.java | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/SecurityBaseRestHandlerTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/SecurityBaseRestHandlerTests.java index b64e619f96bcb..48009f05e22bc 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/SecurityBaseRestHandlerTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/SecurityBaseRestHandlerTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.test.rest.FakeRestChannel; import org.elasticsearch.test.rest.FakeRestRequest; @@ -57,21 +58,22 @@ protected RestChannelConsumer innerPrepareRequest(RestRequest request, NodeClien }; FakeRestRequest fakeRestRequest = new FakeRestRequest(); FakeRestChannel fakeRestChannel = new FakeRestChannel(fakeRestRequest, randomBoolean(), securityDefaultEnabled ? 0 : 1); - NodeClient client = mock(NodeClient.class); - assertFalse(consumerCalled.get()); - verifyZeroInteractions(licenseState); - handler.handleRequest(fakeRestRequest, fakeRestChannel, client); - - verify(licenseState).checkFeature(XPackLicenseState.Feature.SECURITY); - if (securityDefaultEnabled) { - assertTrue(consumerCalled.get()); - assertEquals(0, fakeRestChannel.responses().get()); - assertEquals(0, fakeRestChannel.errors().get()); - } else { + try (NodeClient client = new NoOpNodeClient(this.getTestName())) { assertFalse(consumerCalled.get()); - assertEquals(0, fakeRestChannel.responses().get()); - assertEquals(1, fakeRestChannel.errors().get()); + verifyZeroInteractions(licenseState); + handler.handleRequest(fakeRestRequest, fakeRestChannel, client); + + verify(licenseState).checkFeature(XPackLicenseState.Feature.SECURITY); + if (securityDefaultEnabled) { + assertTrue(consumerCalled.get()); + assertEquals(0, fakeRestChannel.responses().get()); + assertEquals(0, fakeRestChannel.errors().get()); + } else { + assertFalse(consumerCalled.get()); + assertEquals(0, fakeRestChannel.responses().get()); + assertEquals(1, fakeRestChannel.errors().get()); + } } } } From f744cd57d026787b4d27f3983aca3310019313f5 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 18:17:31 -0600 Subject: [PATCH 025/161] Fix NPE/client mock in RestHasPrivilegesActionTests --- .../action/user/RestHasPrivilegesActionTests.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/user/RestHasPrivilegesActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/user/RestHasPrivilegesActionTests.java index 835d44ba28ff7..29fdba89705f0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/user/RestHasPrivilegesActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/user/RestHasPrivilegesActionTests.java @@ -16,6 +16,7 @@ import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.test.rest.FakeRestChannel; import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.xpack.core.security.SecurityContext; @@ -39,13 +40,14 @@ public void testBodyConsumed() throws Exception { final XPackLicenseState licenseState = mock(XPackLicenseState.class); final RestHasPrivilegesAction action = new RestHasPrivilegesAction(Settings.EMPTY, mock(SecurityContext.class), licenseState); - try (XContentBuilder bodyBuilder = JsonXContent.contentBuilder().startObject().endObject()) { + try (XContentBuilder bodyBuilder = JsonXContent.contentBuilder().startObject().endObject(); + NodeClient client = new NoOpNodeClient(this.getTestName())) { final RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) .withPath("/_security/user/_has_privileges/") .withContent(new BytesArray(bodyBuilder.toString()), XContentType.JSON) .build(); final RestChannel channel = new FakeRestChannel(request, true, 1); - action.handleRequest(request, channel, mock(NodeClient.class)); + action.handleRequest(request, channel, client); } } @@ -54,13 +56,14 @@ public void testBasicLicense() throws Exception { final RestHasPrivilegesAction action = new RestHasPrivilegesAction(Settings.EMPTY, mock(SecurityContext.class), licenseState); when(licenseState.checkFeature(XPackLicenseState.Feature.SECURITY)).thenReturn(false); - try (XContentBuilder bodyBuilder = JsonXContent.contentBuilder().startObject().endObject()) { + try (XContentBuilder bodyBuilder = JsonXContent.contentBuilder().startObject().endObject(); + NodeClient client = new NoOpNodeClient(this.getTestName())) { final RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) .withPath("/_security/user/_has_privileges/") .withContent(new BytesArray(bodyBuilder.toString()), XContentType.JSON) .build(); final FakeRestChannel channel = new FakeRestChannel(request, true, 1); - action.handleRequest(request, channel, mock(NodeClient.class)); + action.handleRequest(request, channel, client); assertThat(channel.capturedResponse(), notNullValue()); assertThat(channel.capturedResponse().status(), equalTo(RestStatus.FORBIDDEN)); assertThat( From 853d874c4307bddfd5779ed062dd21b7c94e680c Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 18:20:48 -0600 Subject: [PATCH 026/161] Add allow_system_index_access flag to ML Restart tests as necessary --- .../restart/MlConfigIndexMappingsFullClusterRestartIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java index d6651382e4e4d..4cc09e3bf78e0 100644 --- a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java +++ b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java @@ -92,6 +92,7 @@ public void testMlConfigIndexMappingsAfterMigration() throws Exception { private void assertThatMlConfigIndexDoesNotExist() { Request getIndexRequest = new Request("GET", ".ml-config"); + getIndexRequest.addParameter("allow_system_index_access", "true"); ResponseException e = expectThrows(ResponseException.class, () -> client().performRequest(getIndexRequest)); assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(404)); } @@ -113,6 +114,7 @@ private void createAnomalyDetectorJob(String jobId) throws IOException { @SuppressWarnings("unchecked") private Map getConfigIndexMappings() throws Exception { Request getIndexMappingsRequest = new Request("GET", ".ml-config/_mappings"); + getIndexMappingsRequest.addParameter("allow_system_index_access", "true"); Response getIndexMappingsResponse = client().performRequest(getIndexMappingsRequest); assertThat(getIndexMappingsResponse.getStatusLine().getStatusCode(), equalTo(200)); From 391f1325b59cc7f3e13acfddb396cc8c0be6fba4 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 18:23:45 -0600 Subject: [PATCH 027/161] Add allow_system_index_access flag to cluster restart tests as necessary --- .../elasticsearch/xpack/restart/FullClusterRestartIT.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java index b67dd3c9e9045..88cf9bc5bef6d 100644 --- a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java +++ b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java @@ -100,7 +100,9 @@ public void testSecurityNativeRealm() throws Exception { createRole(true); } else { waitForYellow(".security"); - Response settingsResponse = client().performRequest(new Request("GET", "/.security/_settings/index.format")); + final Request getSettingsRequest = new Request("GET", "/.security/_settings/index.format"); + getSettingsRequest.addParameter("allow_system_index_access", "true"); + Response settingsResponse = client().performRequest(getSettingsRequest); Map settingsResponseMap = entityAsMap(settingsResponse); logger.info("settings response map {}", settingsResponseMap); final String concreteSecurityIndex; @@ -174,7 +176,9 @@ public void testWatcher() throws Exception { logger.info("checking that the Watches index is the correct version"); - Response settingsResponse = client().performRequest(new Request("GET", "/.watches/_settings/index.format")); + final Request getSettingsRequest = new Request("GET", "/.watches/_settings/index.format"); + getSettingsRequest.addParameter("allow_system_index_access", "true"); + Response settingsResponse = client().performRequest(getSettingsRequest); Map settingsResponseMap = entityAsMap(settingsResponse); logger.info("settings response map {}", settingsResponseMap); final String concreteWatchesIndex; From 2fdc53e684ebebe94c141f6c6fa72455c9537825 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 18:53:41 -0600 Subject: [PATCH 028/161] Add allow_system_index_access flag to Transform tests as necessary --- .../TransformConfigurationIndexIT.java | 1 + .../integration/TransformInternalIndexIT.java | 51 ++++++++++++------- .../integration/TransformRobustnessIT.java | 4 +- .../integration/TransformUsageIT.java | 1 + 4 files changed, 37 insertions(+), 20 deletions(-) diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java index dfab389c4e423..38e75662a6861 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java @@ -43,6 +43,7 @@ public void testDeleteConfigurationLeftOver() throws IOException { final StringEntity entity = new StringEntity(Strings.toString(builder), ContentType.APPLICATION_JSON); Request req = new Request("PUT", TransformInternalIndexConstants.LATEST_INDEX_NAME + "/_doc/" + TransformConfig.documentId(fakeTransformName)); + req.addParameter("allow_system_index_access", "true"); req.setEntity(entity); client().performRequest(req); } diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java index 82bed7ac0ea78..9ef5fa1a86316 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java @@ -6,11 +6,10 @@ package org.elasticsearch.xpack.transform.integration; -import org.elasticsearch.action.get.GetRequest; -import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.indices.CreateIndexRequest; import org.elasticsearch.client.transform.GetTransformRequest; @@ -22,7 +21,6 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.search.SearchModule; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.xpack.core.transform.TransformField; @@ -36,7 +34,6 @@ import static org.elasticsearch.xpack.transform.persistence.TransformInternalIndex.addTransformsConfigMappings; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; public class TransformInternalIndexIT extends ESRestTestCase { @@ -79,14 +76,18 @@ public void testUpdateDeletesOldTransformConfig() throws Exception { + " } } } }," + "\"frequency\":\"1s\"" + "}"; - client.index(new IndexRequest(OLD_INDEX) - .id(TransformConfig.documentId(transformId)) - .source(config, XContentType.JSON) - .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE), - RequestOptions.DEFAULT); - GetResponse getResponse = client.get(new GetRequest(OLD_INDEX, TransformConfig.documentId(transformId)), - RequestOptions.DEFAULT); - assertThat(getResponse.isExists(), is(true)); + Request indexRequest = new Request("PUT", OLD_INDEX + "/_doc/" + TransformConfig.documentId(transformId)); + indexRequest.addParameter("allow_system_index_access", "true"); + indexRequest.addParameter("refresh", "true"); + indexRequest.setJsonEntity(config); + assertOK(client().performRequest(indexRequest)); + + { + Request getRequest = new Request("GET", OLD_INDEX + "/_doc/" + TransformConfig.documentId(transformId)); + getRequest.addParameter("allow_system_index_access", "true"); + Response getResponse = client().performRequest(getRequest); + assertOK(getResponse); + } GetTransformResponse response = client.transform() .getTransform(new GetTransformRequest(transformId), RequestOptions.DEFAULT); @@ -100,13 +101,25 @@ public void testUpdateDeletesOldTransformConfig() throws Exception { assertThat(updated.getTransformConfiguration().getDescription(), equalTo("updated")); // Old should now be gone - getResponse = client.get(new GetRequest(OLD_INDEX, TransformConfig.documentId(transformId)), RequestOptions.DEFAULT); - assertThat(getResponse.isExists(), is(false)); + { + Request getRequest = new Request("GET", OLD_INDEX + "/_doc/" + TransformConfig.documentId(transformId)); + getRequest.addParameter("allow_system_index_access", "true"); + try { + Response getResponse = client().performRequest(getRequest); + assertThat(getResponse.getStatusLine().getStatusCode(), equalTo(404)); + } catch (ResponseException e) { + // this is fine, we want it to 404 + assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(404)); + } + } // New should be here - getResponse = client.get(new GetRequest(CURRENT_INDEX, TransformConfig.documentId(transformId)), - RequestOptions.DEFAULT); - assertThat(getResponse.isExists(), is(true)); + { + Request getRequest = new Request("GET", CURRENT_INDEX + "/_doc/" + TransformConfig.documentId(transformId)); + getRequest.addParameter("allow_system_index_access", "true"); + Response getResponse = client().performRequest(getRequest); + assertOK(getResponse); + } } diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformRobustnessIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformRobustnessIT.java index 9d069eb5f9366..3e28b006d8eae 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformRobustnessIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformRobustnessIT.java @@ -117,6 +117,8 @@ private int getNumberOfTransformTasks() throws IOException { } private void beEvilAndDeleteTheTransformIndex() throws IOException { - adminClient().performRequest(new Request("DELETE", TransformInternalIndexConstants.LATEST_INDEX_NAME)); + final Request deleteRequest = new Request("DELETE", TransformInternalIndexConstants.LATEST_INDEX_NAME); + deleteRequest.addParameter("allow_system_index_access", "true"); + adminClient().performRequest(deleteRequest); } } diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java index f6184e734e6d9..bb366daa6a134 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java @@ -62,6 +62,7 @@ public void testUsage() throws Exception { + ":" + TransformStoredDoc.NAME ); + statsExistsRequest.addParameter("allow_system_index_access", "true"); // Verify that we have one stat document assertBusy(() -> { Map hasStatsMap = entityAsMap(client().performRequest(statsExistsRequest)); From 01b1fb3397278771252b4a172966f1fd0fc96254 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 10 Aug 2020 19:41:20 -0600 Subject: [PATCH 029/161] Clean up old comments --- .../org/elasticsearch/cluster/metadata/IndexAbstraction.java | 2 +- .../org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java | 2 +- .../ml/action/TransportStartDataFrameAnalyticsActionTests.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java index 833de1fa979d9..3127887f82d8a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java @@ -221,7 +221,7 @@ public boolean isHidden() { public boolean isSystem() { //G-> This is probably not the best way to tell if an alias is a system-index-alias // this should probably be checked on the alias layer and added as a property in the AliasMetadata - return referenceIndexMetadatas.stream().anyMatch(IndexMetadata::isSystem); // G-> + return referenceIndexMetadatas.stream().anyMatch(IndexMetadata::isSystem); } /** diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java index 54b2a10416c01..8caf78f18b186 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java @@ -99,7 +99,7 @@ public static void createIndexAndAliasIfNecessary(Client client, // The initial index name must be suitable for rollover functionality. String firstConcreteIndex = indexPatternPrefix + "-000001"; String[] concreteIndexNames = - resolver.concreteIndexNames(clusterState, IndicesOptions.lenientExpandOpen(), indexPattern); //G-> THIS GONNA BREAK + resolver.concreteIndexNames(clusterState, IndicesOptions.lenientExpandOpen(), indexPattern); //G-> Does this work as expected? Optional indexPointedByCurrentWriteAlias = clusterState.getMetadata().hasAlias(alias) ? clusterState.getMetadata().getIndicesLookup().get(alias).getIndices().stream().findFirst() : Optional.empty(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java index c68f0d03f171f..009c8da358825 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java @@ -62,7 +62,7 @@ public void testVerifyIndicesPrimaryShardsAreActive() { ClusterState cs = csBuilder.build(); assertThat( TransportStartDataFrameAnalyticsAction.verifyIndicesPrimaryShardsAreActive(cs, - new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), indexName), //G-> This is probably gonna break inside + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), indexName), empty()); metadata = new Metadata.Builder(cs.metadata()); From 612c304991744112045a5a1a268a9a8629a75406 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 11 Aug 2020 14:08:55 -0600 Subject: [PATCH 030/161] Allow Kibana APIs to access system indices by default. --- .../src/main/java/org/elasticsearch/kibana/KibanaPlugin.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/modules/kibana/src/main/java/org/elasticsearch/kibana/KibanaPlugin.java b/modules/kibana/src/main/java/org/elasticsearch/kibana/KibanaPlugin.java index 2d3b13989ffeb..016b3bdfa7e48 100644 --- a/modules/kibana/src/main/java/org/elasticsearch/kibana/KibanaPlugin.java +++ b/modules/kibana/src/main/java/org/elasticsearch/kibana/KibanaPlugin.java @@ -133,6 +133,11 @@ public String getName() { return "kibana_" + super.getName(); } + @Override + protected boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override public List routes() { return super.routes().stream() From 6879de89155924c73f39a5398c5c59f7afbb53c9 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 11 Aug 2020 14:11:14 -0600 Subject: [PATCH 031/161] Fix NPE/client mock in RestGetUserPrivilegesActionTests --- .../rest/action/user/RestGetUserPrivilegesActionTests.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/user/RestGetUserPrivilegesActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/user/RestGetUserPrivilegesActionTests.java index 0b3d9ed355e6c..e5ecaaf436b19 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/user/RestGetUserPrivilegesActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/rest/action/user/RestGetUserPrivilegesActionTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.test.rest.FakeRestChannel; import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.xpack.core.security.SecurityContext; @@ -45,7 +46,9 @@ public void testBasicLicense() throws Exception { when(licenseState.checkFeature(XPackLicenseState.Feature.SECURITY)).thenReturn(false); final FakeRestRequest request = new FakeRestRequest(); final FakeRestChannel channel = new FakeRestChannel(request, true, 1); - action.handleRequest(request, channel, mock(NodeClient.class)); + try (NodeClient nodeClient = new NoOpNodeClient(this.getTestName())) { + action.handleRequest(request, channel, nodeClient); + } assertThat(channel.capturedResponse(), notNullValue()); assertThat(channel.capturedResponse().status(), equalTo(RestStatus.FORBIDDEN)); assertThat(channel.capturedResponse().content().utf8ToString(), containsString("current license is non-compliant for [security]")); From 3924661668e2ed08dbed41410254adb0424f70eb Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 11 Aug 2020 15:26:58 -0600 Subject: [PATCH 032/161] Rename and centralize system index access control header key --- .../cluster/metadata/IndexNameExpressionResolver.java | 3 ++- .../main/java/org/elasticsearch/rest/BaseRestHandler.java | 6 ++++-- .../cluster/metadata/IndexNameExpressionResolverTests.java | 3 ++- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index bacc4e001988d..5ead3aa5166cf 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -62,6 +62,7 @@ public class IndexNameExpressionResolver { public static final String EXCLUDED_DATA_STREAMS_KEY = "es.excluded_ds"; + public static final String SYSTEM_INDEX_ACCESS_CONTROL_KEY = "_prevent_system_index_access"; private final DateMathExpressionResolver dateMathExpressionResolver = new DateMathExpressionResolver(); private final WildcardExpressionResolver wildcardExpressionResolver = new WildcardExpressionResolver(); @@ -668,7 +669,7 @@ private boolean isSystemIndexAccessAllowed() { if (threadContext == null) { return false; } - return "true".equals(threadContext.getHeader("_from_rest")) == false; + return "true".equals(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY)) == false; } public static class Context { diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 63f109ac3cb78..318cad1cde7cd 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -42,6 +42,8 @@ import java.util.concurrent.atomic.LongAdder; import java.util.stream.Collectors; +import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_KEY; + /** * Base handler for REST requests. *

@@ -80,11 +82,11 @@ public final void handleRequest(RestRequest request, RestChannel channel, NodeCl if (allowSystemIndexAccessByDefault() == false) { final String allow_system_index_access = request.param("allow_system_index_access"); final ThreadContext threadContext = client.threadPool().getThreadContext(); - if (threadContext.getHeader("_from_rest") == null + if (threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY) == null && (allow_system_index_access == null || allow_system_index_access.isEmpty() || Boolean.parseBoolean(allow_system_index_access) == false)) { - threadContext.putHeader("_from_rest", "true"); + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, "true"); } } // prepare the request for execution; has the side effect of touching the request parameters 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 88de93ad87dc4..032df4389cdc8 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -55,6 +55,7 @@ import static org.elasticsearch.cluster.DataStreamTestHelper.createBackingIndex; import static org.elasticsearch.cluster.DataStreamTestHelper.createTimestampField; import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_HIDDEN_SETTING; +import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_KEY; import static org.elasticsearch.common.util.set.Sets.newHashSet; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; @@ -1872,7 +1873,7 @@ public void testSystemIndexResolutionWhenAllowed() { public void testSystemIndexResolutionBlocked() { // Set up the thread context to disallow system index access - threadContext.putHeader("_from_rest", "true"); + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, "true"); Settings settings = Settings.builder().build(); Metadata.Builder mdBuilder = Metadata.builder() From 19147b7f2ad8520c524586df0ee9a36d478b9a59 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 11 Aug 2020 15:57:24 -0600 Subject: [PATCH 033/161] Add allow_system_index_access flag to Transform Upgrade tests as necessary --- .../org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java index dfdfd74bfe56d..315302d904bf5 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java @@ -255,6 +255,7 @@ private void awaitWrittenIndexerState(String id, Consumer> responseAss TRANSFORM_INTERNAL_INDEX_PREFIX + "*," + TRANSFORM_INTERNAL_INDEX_PREFIX_DEPRECATED + "*" + "/_search"); + getStatsDocsRequest.addParameter("allow_system_index_access", "true"); getStatsDocsRequest.setJsonEntity("{\n" + " \"query\": {\n" + From b80f5161619ba7b53a3421da639311aeaecf6e51 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 12 Aug 2020 14:45:13 -0600 Subject: [PATCH 034/161] Add allow_system_index_access flag to relevant REST specs --- .../src/main/resources/rest-api-spec/api/cluster.health.json | 4 ++++ .../src/main/resources/rest-api-spec/api/delete_by_query.json | 4 ++++ .../src/main/resources/rest-api-spec/api/field_caps.json | 4 ++++ .../main/resources/rest-api-spec/api/indices.add_block.json | 4 ++++ .../main/resources/rest-api-spec/api/indices.clear_cache.json | 4 ++++ .../src/main/resources/rest-api-spec/api/indices.close.json | 4 ++++ .../src/main/resources/rest-api-spec/api/indices.exists.json | 4 ++++ .../resources/rest-api-spec/api/indices.exists_alias.json | 4 ++++ .../main/resources/rest-api-spec/api/indices.exists_type.json | 4 ++++ .../src/main/resources/rest-api-spec/api/indices.flush.json | 4 ++++ .../main/resources/rest-api-spec/api/indices.forcemerge.json | 4 ++++ .../src/main/resources/rest-api-spec/api/indices.get.json | 4 ++++ .../main/resources/rest-api-spec/api/indices.get_alias.json | 4 ++++ .../rest-api-spec/api/indices.get_field_mapping.json | 4 ++++ .../main/resources/rest-api-spec/api/indices.get_mapping.json | 4 ++++ .../resources/rest-api-spec/api/indices.get_settings.json | 4 ++++ .../src/main/resources/rest-api-spec/api/indices.open.json | 4 ++++ .../main/resources/rest-api-spec/api/indices.put_mapping.json | 4 ++++ .../resources/rest-api-spec/api/indices.put_settings.json | 4 ++++ .../resources/rest-api-spec/api/indices.resolve_index.json | 4 ++++ .../main/resources/rest-api-spec/api/indices.segments.json | 4 ++++ .../resources/rest-api-spec/api/indices.shard_stores.json | 4 ++++ .../src/main/resources/rest-api-spec/api/indices.stats.json | 4 ++++ .../src/main/resources/rest-api-spec/api/indices.upgrade.json | 4 ++++ .../resources/rest-api-spec/api/indices.validate_query.json | 4 ++++ .../src/main/resources/rest-api-spec/api/rank_eval.json | 4 ++++ .../src/main/resources/rest-api-spec/api/search_shards.json | 4 ++++ .../src/main/resources/rest-api-spec/api/update_by_query.json | 4 ++++ 28 files changed, 112 insertions(+) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json index 894b141f2f3b3..09adf14085989 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json @@ -98,6 +98,10 @@ "red" ], "description":"Wait until cluster is in a specific state" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json index 8641c18382dfc..27df42cbdf575 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json @@ -176,6 +176,10 @@ "type":"number|string", "default":1, "description":"The number of slices this task should be divided into. Defaults to 1, meaning the task isn't sliced into subtasks. Can be set to `auto`." + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json index 20a87c7da8883..841b1cb190cf9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json @@ -58,6 +58,10 @@ "type":"boolean", "default":false, "description":"Indicates whether unmapped fields should be included in the response." + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.add_block.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.add_block.json index 7389fb1322824..bf410959be08a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.add_block.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.add_block.json @@ -53,6 +53,10 @@ ], "default":"open", "description":"Whether to expand wildcard expression to concrete indices that are open, closed or both." + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json index 64c10a520c7c4..ff23fe4eba0a9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json @@ -67,6 +67,10 @@ "request":{ "type":"boolean", "description":"Clear request cache" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json index f26c8e77a06a6..71759a2a38e90 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json @@ -53,6 +53,10 @@ "wait_for_active_shards":{ "type":"string", "description":"Sets the number of active shards to wait for before the operation returns." + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json index 7539f44a81eed..c78ef2ab50512 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json @@ -54,6 +54,10 @@ "type":"boolean", "description":"Whether to return all default setting for each of the indices.", "default":false + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json index 66e5ce92cbbe5..9cd4a9ce1a9c0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json @@ -61,6 +61,10 @@ "local":{ "type":"boolean", "description":"Return local information, do not retrieve the state from master node (default: false)" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json index c854d0e8fd841..84357c8782d09 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json @@ -49,6 +49,10 @@ "local":{ "type":"boolean", "description":"Return local information, do not retrieve the state from master node (default: false)" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json index 35138b920466f..276460f43b4b0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json @@ -57,6 +57,10 @@ ], "default":"open", "description":"Whether to expand wildcard expression to concrete indices that are open, closed or both." + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json index 6036b75bb83e4..fd8dca41045b2 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json @@ -59,6 +59,10 @@ "only_expunge_deletes":{ "type":"boolean", "description":"Specify whether the operation should only expunge deleted documents" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json index 90a1274ecb059..455c73a589359 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json @@ -58,6 +58,10 @@ "master_timeout":{ "type":"time", "description":"Specify timeout for connection to master" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json index 5bfdd985a0a85..5eb8049a05750 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json @@ -79,6 +79,10 @@ "local":{ "type":"boolean", "description":"Return local information, do not retrieve the state from master node (default: false)" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json index 0e71b6d395777..29488591a9e19 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json @@ -65,6 +65,10 @@ "local":{ "type":"boolean", "description":"Return local information, do not retrieve the state from master node (default: false)" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json index 24fd668069697..559fc0203c297 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json @@ -59,6 +59,10 @@ "version":"7.8.0", "description":"This parameter is a no-op and field mappings are always retrieved locally." } + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json index 68e325446d3dc..99872624f3425 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json @@ -92,6 +92,10 @@ "type":"boolean", "description":"Whether to return all default setting for each of the indices.", "default":false + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json index 1dab468ce4ff4..98793b6ae2eb6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json @@ -53,6 +53,10 @@ "wait_for_active_shards":{ "type":"string", "description":"Sets the number of active shards to wait for before the operation returns." + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json index 451cbccd8d329..52e0f0215a3bf 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json @@ -55,6 +55,10 @@ "type":"boolean", "default":false, "description":"When true, applies mappings only to the write index of an alias or data stream" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json index 66fe23bab8ba2..736b05618b7bb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json @@ -63,6 +63,10 @@ "flat_settings":{ "type":"boolean", "description":"Return settings in flat format (default: false)" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.resolve_index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.resolve_index.json index 41d609818dbc8..dd72c970c2ed4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.resolve_index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.resolve_index.json @@ -33,6 +33,10 @@ ], "default":"open", "description":"Whether wildcard expressions should get expanded to open or closed indices (default: open)" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json index 83430a9a85600..db4574b974cb4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json @@ -52,6 +52,10 @@ "type":"boolean", "description":"Includes detailed memory usage by Lucene.", "default":false + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json index 7e48e99916171..86896bdca1a8a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json @@ -57,6 +57,10 @@ ], "default":"open", "description":"Whether to expand wildcard expression to concrete indices that are open, closed or both." + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json index 2a3659dfb892a..4576fb9fae848 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json @@ -150,6 +150,10 @@ "type":"boolean", "description":"If set to false stats will also collected from closed indices if explicitly specified or if expand_wildcards expands to closed indices", "default":true + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json index 406fbacda307c..2cf232e28306e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json @@ -63,6 +63,10 @@ "only_ancient_segments":{ "type":"boolean", "description":"If true, only ancient (an older Lucene major release) segments will be upgraded" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json index 3becec003a9e6..1d241fcdeffe5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json @@ -112,6 +112,10 @@ "all_shards":{ "type":"boolean", "description":"Execute validation on all shards instead of one random shard per index" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json b/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json index eadf240192394..1e680d022f232 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json @@ -57,6 +57,10 @@ "dfs_query_then_fetch" ], "description":"Search operation type" + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json index 74b7055b4c4b0..47157edba4ec7 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json @@ -61,6 +61,10 @@ ], "default":"open", "description":"Whether to expand wildcard expression to concrete indices that are open, closed or both." + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json index 86fa8a60db43b..dbd5773ead331 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json @@ -184,6 +184,10 @@ "type":"number|string", "default":1, "description":"The number of slices this task should be divided into. Defaults to 1, meaning the task isn't sliced into subtasks. Can be set to `auto`." + }, + "allow_system_index_access": { + "type":"boolean", + "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ From 71da5f85678db243c482609e60e5e7b1e7521f37 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 12 Aug 2020 14:53:36 -0600 Subject: [PATCH 035/161] ClusterModule now requires a ThreadContext instead of a ThreadPool --- .../org/elasticsearch/cluster/ClusterModule.java | 8 ++++---- .../src/main/java/org/elasticsearch/node/Node.java | 3 ++- .../elasticsearch/cluster/ClusterModuleTests.java | 12 ++++++------ 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java index 4d2aeef2919a8..8b932077603bc 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java @@ -23,10 +23,10 @@ import org.elasticsearch.cluster.action.index.NodeMappingRefreshAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.metadata.ComponentTemplateMetadata; +import org.elasticsearch.cluster.metadata.ComposableIndexTemplateMetadata; import org.elasticsearch.cluster.metadata.DataStreamMetadata; import org.elasticsearch.cluster.metadata.IndexGraveyard; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.metadata.ComposableIndexTemplateMetadata; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.MetadataDeleteIndexService; import org.elasticsearch.cluster.metadata.MetadataIndexAliasesService; @@ -68,6 +68,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.ingest.IngestMetadata; @@ -77,7 +78,6 @@ import org.elasticsearch.script.ScriptMetadata; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskResultsService; -import org.elasticsearch.threadpool.ThreadPool; import java.util.ArrayList; import java.util.Collection; @@ -108,13 +108,13 @@ public class ClusterModule extends AbstractModule { final ShardsAllocator shardsAllocator; public ClusterModule(Settings settings, ClusterService clusterService, List clusterPlugins, - ClusterInfoService clusterInfoService, ThreadPool threadPool) { + ClusterInfoService clusterInfoService, ThreadContext threadContext) { this.clusterPlugins = clusterPlugins; this.deciderList = createAllocationDeciders(settings, clusterService.getClusterSettings(), clusterPlugins); this.allocationDeciders = new AllocationDeciders(deciderList); this.shardsAllocator = createShardsAllocator(settings, clusterService.getClusterSettings(), clusterPlugins); this.clusterService = clusterService; - this.indexNameExpressionResolver = new IndexNameExpressionResolver(threadPool.getThreadContext()); + this.indexNameExpressionResolver = new IndexNameExpressionResolver(threadContext); this.allocationService = new AllocationService(allocationDeciders, shardsAllocator, clusterInfoService); } diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index e15b8c264df4b..4e8f3b90ff959 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -395,7 +395,8 @@ protected Node(final Environment initialEnvironment, final MonitorService monitorService = new MonitorService(settings, nodeEnvironment, threadPool); final FsHealthService fsHealthService = new FsHealthService(settings, clusterService.getClusterSettings(), threadPool, nodeEnvironment); - ClusterModule clusterModule = new ClusterModule(settings, clusterService, clusterPlugins, clusterInfoService, threadPool); + ClusterModule clusterModule = new ClusterModule(settings, clusterService, clusterPlugins, clusterInfoService, + threadPool.getThreadContext()); modules.add(clusterModule); IndicesModule indicesModule = new IndicesModule(pluginsService.filterPlugins(MapperPlugin.class)); modules.add(indicesModule); diff --git a/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java b/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java index 12a291f107116..9e8afca134329 100644 --- a/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java @@ -137,7 +137,7 @@ public void testRegisterAllocationDeciderDuplicate() { public Collection createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) { return Collections.singletonList(new EnableAllocationDecider(settings, clusterSettings)); } - }), clusterInfoService, threadPool)); + }), clusterInfoService, threadPool.getThreadContext())); assertEquals(e.getMessage(), "Cannot specify allocation decider [" + EnableAllocationDecider.class.getName() + "] twice"); } @@ -149,7 +149,7 @@ public void testRegisterAllocationDecider() { public Collection createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) { return Collections.singletonList(new FakeAllocationDecider()); } - }), clusterInfoService, threadPool); + }), clusterInfoService, threadPool.getThreadContext()); assertTrue(module.deciderList.stream().anyMatch(d -> d.getClass().equals(FakeAllocationDecider.class))); } @@ -161,7 +161,7 @@ public Map> getShardsAllocators(Settings setti return Collections.singletonMap(name, supplier); } } - ), clusterInfoService, threadPool); + ), clusterInfoService, threadPool.getThreadContext()); } public void testRegisterShardsAllocator() { @@ -179,7 +179,7 @@ public void testRegisterShardsAllocatorAlreadyRegistered() { public void testUnknownShardsAllocator() { Settings settings = Settings.builder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING.getKey(), "dne").build(); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> - new ClusterModule(settings, clusterService, Collections.emptyList(), clusterInfoService, threadPool)); + new ClusterModule(settings, clusterService, Collections.emptyList(), clusterInfoService, threadPool.getThreadContext())); assertEquals("Unknown ShardsAllocator [dne]", e.getMessage()); } @@ -222,14 +222,14 @@ public void testAllocationDeciderOrder() { public void testRejectsReservedExistingShardsAllocatorName() { final ClusterModule clusterModule = new ClusterModule(Settings.EMPTY, clusterService, - List.of(existingShardsAllocatorPlugin(GatewayAllocator.ALLOCATOR_NAME)), clusterInfoService, threadPool); + List.of(existingShardsAllocatorPlugin(GatewayAllocator.ALLOCATOR_NAME)), clusterInfoService, threadPool.getThreadContext()); expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator())); } public void testRejectsDuplicateExistingShardsAllocatorName() { final ClusterModule clusterModule = new ClusterModule(Settings.EMPTY, clusterService, List.of(existingShardsAllocatorPlugin("duplicate"), existingShardsAllocatorPlugin("duplicate")), - clusterInfoService, threadPool); + clusterInfoService, threadPool.getThreadContext()); expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator())); } From 9e5b251e53303e3699fa773555cddb154ea21dec Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 12 Aug 2020 14:54:18 -0600 Subject: [PATCH 036/161] Remove unnecessary null check in IndexNameExpressionResolver --- .../cluster/metadata/IndexNameExpressionResolver.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 5ead3aa5166cf..6efc946930de4 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -666,9 +666,6 @@ boolean isPatternMatchingAllIndices(Metadata metadata, String[] indicesOrAliases } private boolean isSystemIndexAccessAllowed() { - if (threadContext == null) { - return false; - } return "true".equals(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY)) == false; } From b4128e75765aa565a91aecb8f1d2804d8d0c240a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 12 Aug 2020 15:02:40 -0600 Subject: [PATCH 037/161] s/Boolean.parseBoolean/Booleans.parseBoolean --- .../main/java/org/elasticsearch/rest/BaseRestHandler.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 318cad1cde7cd..1477c9ecdf1b9 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -22,6 +22,7 @@ import org.apache.lucene.search.spell.LevenshteinDistance; import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.Booleans; import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Setting; @@ -83,9 +84,7 @@ public final void handleRequest(RestRequest request, RestChannel channel, NodeCl final String allow_system_index_access = request.param("allow_system_index_access"); final ThreadContext threadContext = client.threadPool().getThreadContext(); if (threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY) == null - && (allow_system_index_access == null - || allow_system_index_access.isEmpty() - || Boolean.parseBoolean(allow_system_index_access) == false)) { + && (Booleans.parseBoolean(allow_system_index_access, false) == false)) { threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, "true"); } } From 948e2bc762c7eebd02a876a092da9229bcf5b354 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 12 Aug 2020 15:03:14 -0600 Subject: [PATCH 038/161] Remove accidental double space per review --- .../org/elasticsearch/action/support/AutoCreateIndexTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/action/support/AutoCreateIndexTests.java b/server/src/test/java/org/elasticsearch/action/support/AutoCreateIndexTests.java index ff20c4e05fd7f..87b88e8a818d8 100644 --- a/server/src/test/java/org/elasticsearch/action/support/AutoCreateIndexTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/AutoCreateIndexTests.java @@ -178,7 +178,7 @@ public void testUpdate() { ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - AutoCreateIndex autoCreateIndex = new AutoCreateIndex(settings, clusterSettings, + AutoCreateIndex autoCreateIndex = new AutoCreateIndex(settings, clusterSettings, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); assertThat(autoCreateIndex.getAutoCreate().isAutoCreateIndex(), equalTo(value)); From 0bc64f60471f11feaac927e8099f7243f032ef53 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 12 Aug 2020 15:05:33 -0600 Subject: [PATCH 039/161] Ensure method is actually called in RestBulkActionTests --- .../action/document/RestBulkActionTests.java | 27 +++++++++---------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java index 550d8075e27da..c28bf08b27dc5 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.rest.action.document; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.bulk.BulkRequest; @@ -46,7 +47,16 @@ public class RestBulkActionTests extends ESTestCase { public void testBulkPipelineUpsert() throws Exception { - try (NodeClient verifyingClient = new VerifyingClient(this.getTestName())) { + SetOnce bulkCalled = new SetOnce<>(); + try (NodeClient verifyingClient = new NoOpNodeClient(this.getTestName()) { + @Override + public void bulk(BulkRequest request, ActionListener listener) { + bulkCalled.set(true); + assertThat(request.requests(), hasSize(2)); + UpdateRequest updateRequest = (UpdateRequest) request.requests().get(1); + assertThat(updateRequest.upsertRequest().getPipeline(), equalTo("timestamps")); + } + }) { final Map params = new HashMap<>(); params.put("pipeline", "timestamps"); new RestBulkAction(settings(Version.CURRENT).build()) @@ -64,20 +74,7 @@ public void testBulkPipelineUpsert() throws Exception { ).withMethod(RestRequest.Method.POST).build(), mock(RestChannel.class), verifyingClient ); - - } - } - - private static class VerifyingClient extends NoOpNodeClient { - VerifyingClient(String testName) { - super(testName); - } - - @Override - public void bulk(BulkRequest request, ActionListener listener) { - assertThat(request.requests(), hasSize(2)); - UpdateRequest updateRequest = (UpdateRequest) request.requests().get(1); - assertThat(updateRequest.upsertRequest().getPipeline(), equalTo("timestamps")); + assertThat(bulkCalled.get(), equalTo(true)); } } } From 8d97b0fefb65ef05cd6a73414ea4b6ea5bc82063 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 12 Aug 2020 15:08:59 -0600 Subject: [PATCH 040/161] Move allow_system key to constant & add version constant --- .../main/java/org/elasticsearch/rest/BaseRestHandler.java | 5 ++++- .../java/org/elasticsearch/test/rest/ESRestTestCase.java | 3 ++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 1477c9ecdf1b9..a19797993122b 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -21,6 +21,7 @@ import org.apache.lucene.search.spell.LevenshteinDistance; import org.apache.lucene.util.CollectionUtil; +import org.elasticsearch.Version; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.CheckedConsumer; @@ -57,6 +58,8 @@ public abstract class BaseRestHandler implements RestHandler { public static final Setting MULTI_ALLOW_EXPLICIT_INDEX = Setting.boolSetting("rest.action.multi.allow_explicit_index", true, Property.NodeScope); + public static final Version ALLOW_SYSTEM_INDEX_ADDED_VERSION = Version.V_8_0_0; + public static final String ALLOW_SYSTEM_INDEX_ACCESS_KEY = "allow_system_index_access"; private final LongAdder usageCount = new LongAdder(); @@ -81,7 +84,7 @@ public final long getUsageCount() { @Override public final void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { if (allowSystemIndexAccessByDefault() == false) { - final String allow_system_index_access = request.param("allow_system_index_access"); + final String allow_system_index_access = request.param(ALLOW_SYSTEM_INDEX_ACCESS_KEY); final ThreadContext threadContext = client.threadPool().getThreadContext(); if (threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY) == null && (Booleans.parseBoolean(allow_system_index_access, false) == false)) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 28c8f9be23701..e6372042b373a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -97,6 +97,7 @@ import static java.util.Collections.sort; import static java.util.Collections.unmodifiableList; +import static org.elasticsearch.rest.BaseRestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; import static org.hamcrest.Matchers.anEmptyMap; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; @@ -642,7 +643,7 @@ private void wipeCluster() throws Exception { protected static void wipeAllIndices() throws IOException { boolean includeHidden = minimumNodeVersion().onOrAfter(Version.V_7_7_0); - boolean includeSystem = minimumNodeVersion().onOrAfter(Version.V_8_0_0); + boolean includeSystem = minimumNodeVersion().onOrAfter(ALLOW_SYSTEM_INDEX_ADDED_VERSION); try { final Request deleteRequest = new Request("DELETE", "*"); deleteRequest.addParameter("expand_wildcards", "open,closed" + (includeHidden ? ",hidden" : "")); From e16b6dc13104d69e3c86198b8c5723a86e7633ac Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 12 Aug 2020 15:30:15 -0600 Subject: [PATCH 041/161] Refactor ThreadContext header to use positive logic --- .../cluster/metadata/IndexNameExpressionResolver.java | 5 +++-- .../main/java/org/elasticsearch/rest/BaseRestHandler.java | 6 +++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 6efc946930de4..dafb785ae2160 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -62,7 +63,7 @@ public class IndexNameExpressionResolver { public static final String EXCLUDED_DATA_STREAMS_KEY = "es.excluded_ds"; - public static final String SYSTEM_INDEX_ACCESS_CONTROL_KEY = "_prevent_system_index_access"; + public static final String SYSTEM_INDEX_ACCESS_CONTROL_KEY = "_system_index_access_allowed"; private final DateMathExpressionResolver dateMathExpressionResolver = new DateMathExpressionResolver(); private final WildcardExpressionResolver wildcardExpressionResolver = new WildcardExpressionResolver(); @@ -666,7 +667,7 @@ boolean isPatternMatchingAllIndices(Metadata metadata, String[] indicesOrAliases } private boolean isSystemIndexAccessAllowed() { - return "true".equals(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY)) == false; + return Booleans.parseBoolean(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY), true); } public static class Context { diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index a19797993122b..ae398262cc544 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -84,11 +84,11 @@ public final long getUsageCount() { @Override public final void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { if (allowSystemIndexAccessByDefault() == false) { - final String allow_system_index_access = request.param(ALLOW_SYSTEM_INDEX_ACCESS_KEY); + final String allowSystemIndexParameter = request.param(ALLOW_SYSTEM_INDEX_ACCESS_KEY); final ThreadContext threadContext = client.threadPool().getThreadContext(); if (threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY) == null - && (Booleans.parseBoolean(allow_system_index_access, false) == false)) { - threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, "true"); + && Booleans.parseBoolean(allowSystemIndexParameter, false)) { + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, Boolean.FALSE.toString()); } } // prepare the request for execution; has the side effect of touching the request parameters From 3e979d9e42fbdb087d6beb4742f85c9a1aca2345 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 12 Aug 2020 15:47:11 -0600 Subject: [PATCH 042/161] Fix test after inverting header logic --- .../cluster/metadata/IndexNameExpressionResolverTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 032df4389cdc8..4e9b4b60e167f 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -1873,7 +1873,7 @@ public void testSystemIndexResolutionWhenAllowed() { public void testSystemIndexResolutionBlocked() { // Set up the thread context to disallow system index access - threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, "true"); + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, Boolean.FALSE.toString()); Settings settings = Settings.builder().build(); Metadata.Builder mdBuilder = Metadata.builder() From 4a8143804ef8f857edb4eca585c555de4a755b8b Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 13 Aug 2020 09:34:54 -0600 Subject: [PATCH 043/161] Fix logic when attaching header --- .../src/main/java/org/elasticsearch/rest/BaseRestHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index ae398262cc544..31cbd7de3421a 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -87,7 +87,7 @@ public final void handleRequest(RestRequest request, RestChannel channel, NodeCl final String allowSystemIndexParameter = request.param(ALLOW_SYSTEM_INDEX_ACCESS_KEY); final ThreadContext threadContext = client.threadPool().getThreadContext(); if (threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY) == null - && Booleans.parseBoolean(allowSystemIndexParameter, false)) { + && Booleans.parseBoolean(allowSystemIndexParameter, false) == false) { threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, Boolean.FALSE.toString()); } } From e708dcf8353ee1ee394ec8567b98d18657369283 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 10:48:01 -0600 Subject: [PATCH 044/161] Fix compilation from merge --- .../xpack/ml/inference/TrainedModelStatsServiceTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/TrainedModelStatsServiceTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/TrainedModelStatsServiceTests.java index f07729e02fdf7..d417019d3f158 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/TrainedModelStatsServiceTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/TrainedModelStatsServiceTests.java @@ -137,7 +137,7 @@ public void testVerifyIndicesExistAndPrimaryShardsAreActive() { public void testUpdateStatsUpgradeMode() { String aliasName = MlStatsIndex.writeAlias(); String concreteIndex = ".ml-stats-000001"; - IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); + IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); // create a valid index routing so persistence will occur RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); From c8588ccb12cba8a053557743f6709ba30075335d Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 13:48:27 -0600 Subject: [PATCH 045/161] Add `allow_system_index_access` flag to test --- .../test/upgraded_cluster/80_transform_jobs_crud.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_transform_jobs_crud.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_transform_jobs_crud.yml index 8e4a540a92cf6..44ff08ce1a2fd 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_transform_jobs_crud.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_transform_jobs_crud.yml @@ -297,6 +297,7 @@ setup: - do: indices.get_mapping: index: .transform-internal-005 + allow_system_index_access: true - match: { \.transform-internal-005.mappings.dynamic: "false" } - match: { \.transform-internal-005.mappings.properties.id.type: "keyword" } - do: From 5325e0a5f47c94b8967022145cc30e259085b296 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 13:56:08 -0600 Subject: [PATCH 046/161] Remove unnecessary comment --- .../org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java index 8caf78f18b186..6b969a5ed1763 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/utils/MlIndexAndAlias.java @@ -99,7 +99,7 @@ public static void createIndexAndAliasIfNecessary(Client client, // The initial index name must be suitable for rollover functionality. String firstConcreteIndex = indexPatternPrefix + "-000001"; String[] concreteIndexNames = - resolver.concreteIndexNames(clusterState, IndicesOptions.lenientExpandOpen(), indexPattern); //G-> Does this work as expected? + resolver.concreteIndexNames(clusterState, IndicesOptions.lenientExpandOpen(), indexPattern); Optional indexPointedByCurrentWriteAlias = clusterState.getMetadata().hasAlias(alias) ? clusterState.getMetadata().getIndicesLookup().get(alias).getIndices().stream().findFirst() : Optional.empty(); From f69bc2b5c9afe97b0a97afbe27f1796b0d1fe0b9 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 14:08:10 -0600 Subject: [PATCH 047/161] INER tests for total wildcards --- .../IndexNameExpressionResolverTests.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) 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 4e9b4b60e167f..d7690cd08157a 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -1869,6 +1869,17 @@ public void testSystemIndexResolutionWhenAllowed() { .collect(Collectors.toList()); assertThat(indexNames, containsInAnyOrder(".watches")); } + + // Full wildcard + { + SearchRequest request = new SearchRequest(randomFrom("*", "_all")); + + List indexNames = Arrays + .stream(indexNameExpressionResolver.concreteIndices(state, request)) + .map(i -> i.getName()) + .collect(Collectors.toList()); + assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); + } } public void testSystemIndexResolutionBlocked() { @@ -1915,6 +1926,17 @@ public void testSystemIndexResolutionBlocked() { .collect(Collectors.toList()); assertThat(indexNames, empty()); } + + // Full wildcard + { + SearchRequest request = new SearchRequest(randomFrom("*", "_all")); + + List indexNames = Arrays + .stream(indexNameExpressionResolver.concreteIndices(state, request)) + .map(i -> i.getName()) + .collect(Collectors.toList()); + assertThat(indexNames, containsInAnyOrder("some-other-index")); + } } public void testDataStreams() { From 5852572050e03bdb6976336a0ad8575c256a66de Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 15:22:37 -0600 Subject: [PATCH 048/161] Add dedicated system index blocking/access integration test --- .../elasticsearch/http/SystemIndexRestIT.java | 164 ++++++++++++++++++ 1 file changed, 164 insertions(+) create mode 100644 qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java new file mode 100644 index 0000000000000..ef8b84d18e1c9 --- /dev/null +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -0,0 +1,164 @@ +/* + * 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. + */ + +package org.elasticsearch.http; + +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.SettingsFilter; +import org.elasticsearch.indices.SystemIndexDescriptor; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.SystemIndexPlugin; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.RestStatusToXContentListener; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +import static org.elasticsearch.test.rest.ESRestTestCase.entityAsMap; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.is; + +public class SystemIndexRestIT extends HttpSmokeTestCase { + + @Override + protected Collection> nodePlugins() { + List> plugins = new ArrayList<>(super.nodePlugins()); + plugins.add(SystemIndexTestPlugin.class); + return plugins; + } + + public void testSystemIndexAccessBlockedByDefault() throws Exception { + // create index + { + Request putDocRequest = new Request("POST", "/_sys_index_test/add_doc/42"); + Response resp = getRestClient().performRequest(putDocRequest); + assertThat(resp.getStatusLine().getStatusCode(), equalTo(201)); + } + + + // make sure the system index now exists (with allow_system_index_access flag) + assertBusy(() -> { + Request searchRequest = new Request("GET", "/" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + "/_count"); + searchRequest.addParameter("allow_system_index_access", "true"); + + // Disallow no indices to cause an exception if the flag above doesn't work + searchRequest.addParameter("allow_no_indices", "false"); + searchRequest.setJsonEntity("{\"query\": {\"match\": {\"some_field\": \"some_value\"}}}"); + + final Response searchResponse = getRestClient().performRequest(searchRequest); + assertThat(searchResponse.getStatusLine().getStatusCode(), is(200)); + Map responseMap = entityAsMap(searchResponse); + assertThat(responseMap, hasKey("count")); + assertThat(responseMap.get("count"), equalTo(1)); + }); + + // now try without `allow_system_index_access` + assertAccessBlocked(SystemIndexTestPlugin.SYSTEM_INDEX_NAME); + + // And with a partial wildcard + assertAccessBlocked(".test-*"); + + // And with a total wildcard + assertAccessBlocked(randomFrom("*", "_all")); + + // Try to index a doc directly + { + Request putDocDirectlyRequest = new Request("PUT", "/" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + "/_doc/43"); + putDocDirectlyRequest.setJsonEntity("{\"some_field\": \"some_other_value\"}"); + ResponseException exception = expectThrows(ResponseException.class, + () -> getRestClient().performRequest(putDocDirectlyRequest)); + logger.info("{}", entityAsMap(exception.getResponse())); + assertThat(exception.getResponse().getStatusLine().getStatusCode(), equalTo(404)); + } + } + + private void assertAccessBlocked(String s) { + Request searchRequest = new Request("GET", "/" + s + randomFrom("/_count", "/_search")); + searchRequest.setJsonEntity("{\"query\": {\"match\": {\"some_field\": \"some_value\"}}}"); + // Disallow no indices to cause an exception if this resolves to zero indices (as we expect) + searchRequest.addParameter("allow_no_indices", "false"); + + ResponseException exception = expectThrows(ResponseException.class, () -> getRestClient().performRequest(searchRequest)); + assertThat(exception.getResponse().getStatusLine().getStatusCode(), equalTo(404)); + } + + public static class SystemIndexTestPlugin extends Plugin implements SystemIndexPlugin { + + public static final String SYSTEM_INDEX_NAME = ".test-system-idx"; + + @Override + public List getRestHandlers(Settings settings, RestController restController, ClusterSettings clusterSettings, + IndexScopedSettings indexScopedSettings, SettingsFilter settingsFilter, + IndexNameExpressionResolver indexNameExpressionResolver, + Supplier nodesInCluster) { + return List.of(new AddDocRestHandler()); + } + + @Override + public Collection getSystemIndexDescriptors(Settings settings) { + return Collections.singletonList(new SystemIndexDescriptor(SYSTEM_INDEX_NAME, "System indices for tests")); + } + + public static class AddDocRestHandler extends BaseRestHandler { + @Override + protected boolean allowSystemIndexAccessByDefault() { + return true; + } + + @Override + public String getName() { + return "system_index_test_doc_adder"; + } + + @Override + public List routes() { + return List.of(new Route(RestRequest.Method.POST, "/_sys_index_test/add_doc/{id}")); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { + IndexRequest indexRequest = new IndexRequest(SYSTEM_INDEX_NAME); + indexRequest.id(request.param("id")); + indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + indexRequest.source(Map.of("some_field", "some_value")); + return channel -> client.index(indexRequest, + new RestStatusToXContentListener<>(channel, r -> r.getLocation(indexRequest.routing()))); + } + } + } +} From 8cadfcaf2651c1ad88e13889dacad3f5292b5755 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 17:26:14 -0600 Subject: [PATCH 049/161] Rules for aliases & system indices + unit tests --- .../cluster/metadata/IndexAbstraction.java | 27 +++++- .../metadata/IndexNameExpressionResolver.java | 2 + .../metadata/IndexAbstractionTests.java | 85 +++++++++++++++++-- 3 files changed, 104 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java index 3127887f82d8a..216495a6498a4 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexAbstraction.java @@ -219,9 +219,7 @@ public boolean isHidden() { @Override public boolean isSystem() { - //G-> This is probably not the best way to tell if an alias is a system-index-alias - // this should probably be checked on the alias layer and added as a property in the AliasMetadata - return referenceIndexMetadatas.stream().anyMatch(IndexMetadata::isSystem); + return referenceIndexMetadatas.stream().allMatch(IndexMetadata::isSystem); } /** @@ -289,6 +287,29 @@ public void computeAndValidateAliasProperties() { Strings.collectionToCommaDelimitedString(nonHiddenOn) + "]; alias must have the same is_hidden setting " + "on all indices"); } + + // Validate system status + + final Map> groupedBySystemStatus = referenceIndexMetadatas.stream() + .collect(Collectors.groupingBy(IndexMetadata::isSystem)); + // If the alias has either all system or all non-system, then no more validation is required + if (isNonEmpty(groupedBySystemStatus.get(false)) && isNonEmpty(groupedBySystemStatus.get(true))) { + final List newVersionSystemIndices = groupedBySystemStatus.get(true).stream() + .filter(i -> i.getCreationVersion().onOrAfter(IndexNameExpressionResolver.SYSTEM_INDEX_ENFORCEMENT_VERSION)) + .map(i -> i.getIndex().getName()) + .sorted() // reliable error message for testing + .collect(Collectors.toList()); + + if (newVersionSystemIndices.isEmpty() == false) { + final List nonSystemIndices = groupedBySystemStatus.get(false).stream() + .map(i -> i.getIndex().getName()) + .sorted() // reliable error message for testing + .collect(Collectors.toList()); + throw new IllegalStateException("alias [" + aliasName + "] refers to both system indices " + newVersionSystemIndices + + " and non-system indices: " + nonSystemIndices + ", but aliases must refer to either system or" + + " non-system indices, not both"); + } + } } private boolean isNonEmpty(List idxMetas) { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index dafb785ae2160..3c91981a40429 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -20,6 +20,7 @@ package org.elasticsearch.cluster.metadata; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.Version; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterState; @@ -64,6 +65,7 @@ public class IndexNameExpressionResolver { public static final String EXCLUDED_DATA_STREAMS_KEY = "es.excluded_ds"; public static final String SYSTEM_INDEX_ACCESS_CONTROL_KEY = "_system_index_access_allowed"; + public static final Version SYSTEM_INDEX_ENFORCEMENT_VERSION = Version.V_8_0_0; private final DateMathExpressionResolver dateMathExpressionResolver = new DateMathExpressionResolver(); private final WildcardExpressionResolver wildcardExpressionResolver = new WildcardExpressionResolver(); diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexAbstractionTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexAbstractionTests.java index ca8033cbfdfe8..327de108ffe28 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexAbstractionTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexAbstractionTests.java @@ -24,8 +24,10 @@ import org.elasticsearch.Version; import org.elasticsearch.common.Nullable; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.VersionUtils; import java.util.Objects; +import java.util.Random; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; @@ -36,12 +38,12 @@ public void testHiddenAliasValidation() { final String hiddenAliasName = "hidden_alias"; AliasMetadata hiddenAliasMetadata = new AliasMetadata.Builder(hiddenAliasName).isHidden(true).build(); - IndexMetadata hidden1 = buildIndexWithAlias("hidden1", hiddenAliasName, true); - IndexMetadata hidden2 = buildIndexWithAlias("hidden2", hiddenAliasName, true); - IndexMetadata hidden3 = buildIndexWithAlias("hidden3", hiddenAliasName, true); + IndexMetadata hidden1 = buildIndexWithAlias("hidden1", hiddenAliasName, true, Version.CURRENT, false); + IndexMetadata hidden2 = buildIndexWithAlias("hidden2", hiddenAliasName, true, Version.CURRENT, false); + IndexMetadata hidden3 = buildIndexWithAlias("hidden3", hiddenAliasName, true, Version.CURRENT, false); - IndexMetadata indexWithNonHiddenAlias = buildIndexWithAlias("nonhidden1", hiddenAliasName, false); - IndexMetadata indexWithUnspecifiedAlias = buildIndexWithAlias("nonhidden2", hiddenAliasName, null); + IndexMetadata indexWithNonHiddenAlias = buildIndexWithAlias("nonhidden1", hiddenAliasName, false, Version.CURRENT, false); + IndexMetadata indexWithUnspecifiedAlias = buildIndexWithAlias("nonhidden2", hiddenAliasName, null, Version.CURRENT, false); { IndexAbstraction.Alias allHidden = new IndexAbstraction.Alias(hiddenAliasMetadata, hidden1); @@ -116,13 +118,82 @@ public void testHiddenAliasValidation() { } } - private IndexMetadata buildIndexWithAlias(String indexName, String aliasName, @Nullable Boolean aliasIsHidden) { + public void testSystemAliasValidation() { + final Random random = random(); + final String systemAlias = "system_alias"; + final Version random7xVersion = VersionUtils.randomVersionBetween(random, Version.V_7_0_0, + VersionUtils.getPreviousVersion(Version.V_8_0_0)); + final AliasMetadata aliasMetadata = new AliasMetadata.Builder(systemAlias).build(); + + final IndexMetadata currentVersionSystem = buildIndexWithAlias(".system1", systemAlias, null, Version.CURRENT, true); + final IndexMetadata currentVersionSystem2 = buildIndexWithAlias(".system2", systemAlias, null, Version.CURRENT, true); + final IndexMetadata oldVersionSystem = buildIndexWithAlias(".oldVersionSystem", systemAlias, null, random7xVersion, true); + + final IndexMetadata regularIndex = buildIndexWithAlias("regular1", systemAlias, false, Version.CURRENT, false); + + // All system on 8.0+ + { + IndexAbstraction.Alias allSystemCurrent = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + allSystemCurrent.addIndex(currentVersionSystem2); + allSystemCurrent.computeAndValidateAliasProperties(); // Should be ok + } + + // All system, some from an older version + { + IndexAbstraction.Alias allSystemMixed = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + allSystemMixed.addIndex(currentVersionSystem2); + allSystemMixed.addIndex(oldVersionSystem); + allSystemMixed.computeAndValidateAliasProperties(); // Should be ok + } + + // All regular + { + IndexAbstraction.Alias allRegular = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + allRegular.addIndex(currentVersionSystem2); + allRegular.addIndex(oldVersionSystem); + allRegular.computeAndValidateAliasProperties(); // Should be ok + } + + // System index from pre-8.0 + regular index should be fine + { + IndexAbstraction.Alias oldAndRegular = new IndexAbstraction.Alias(aliasMetadata, oldVersionSystem); + oldAndRegular.addIndex(regularIndex); + oldAndRegular.computeAndValidateAliasProperties(); // Should be ok + } + + // Current version system index + regular index should fail + { + IndexAbstraction.Alias systemAndRegular = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + systemAndRegular.addIndex(regularIndex); + IllegalStateException exception = expectThrows(IllegalStateException.class, + () -> systemAndRegular.computeAndValidateAliasProperties()); + assertThat(exception.getMessage(), containsString("alias [" + systemAlias + + "] refers to both system indices [" + currentVersionSystem.getIndex().getName() + "] and non-system indices: [" + + regularIndex.getIndex().getName() + "], but aliases must refer to either system or non-system indices, not both")); + } + + // Mixed version system indices + regular index should fail + { + IndexAbstraction.Alias mixedVersionSystemAndRegular = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + mixedVersionSystemAndRegular.addIndex(oldVersionSystem); + mixedVersionSystemAndRegular.addIndex(regularIndex); + IllegalStateException exception = expectThrows(IllegalStateException.class, + () -> mixedVersionSystemAndRegular.computeAndValidateAliasProperties()); + assertThat(exception.getMessage(), containsString("alias [" + systemAlias + + "] refers to both system indices [" + currentVersionSystem.getIndex().getName() + "] and non-system indices: [" + + regularIndex.getIndex().getName() + "], but aliases must refer to either system or non-system indices, not both")); + } + } + + private IndexMetadata buildIndexWithAlias(String indexName, String aliasName, @Nullable Boolean aliasIsHidden, + Version indexCreationVersion, boolean isSystem) { final AliasMetadata.Builder aliasMetadata = new AliasMetadata.Builder(aliasName); if (Objects.nonNull(aliasIsHidden) || randomBoolean()) { aliasMetadata.isHidden(aliasIsHidden); } return new IndexMetadata.Builder(indexName) - .settings(settings(Version.CURRENT)) + .settings(settings(indexCreationVersion)) + .system(isSystem) .numberOfShards(1) .numberOfReplicas(0) .putAlias(aliasMetadata) From c962c240d3dfa1978b13c1689197a5fb55216ee1 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 18:07:45 -0600 Subject: [PATCH 050/161] Clean up system index upgrade tests --- .../upgrades/FullClusterRestartIT.java | 118 +++++++++--------- .../upgrades/SystemIndicesUpgradeIT.java | 109 ++++++++-------- 2 files changed, 108 insertions(+), 119 deletions(-) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index 2272b2eb4092e..cdbda0564bf0c 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -1385,67 +1385,63 @@ public void testResize() throws Exception { } } - public void testCreateSystemIndexInOldVersion() throws Exception { - assumeTrue("only run on old cluster", isRunningAgainstOldCluster()); - // create index - Request createTestIndex = new Request("PUT", "/test_index_old"); - createTestIndex.setJsonEntity("{\"settings\": {\"index.number_of_replicas\": 0}}"); - client().performRequest(createTestIndex); - - Request bulk = new Request("POST", "/_bulk"); - bulk.addParameter("refresh", "true"); - bulk.setJsonEntity("{\"index\": {\"_index\": \"test_index_old\"}}\n" + - "{\"f1\": \"v1\", \"f2\": \"v2\"}\n"); - client().performRequest(bulk); - - // start a async reindex job - Request reindex = new Request("POST", "/_reindex"); - reindex.setJsonEntity( - "{\n" + - " \"source\":{\n" + - " \"index\":\"test_index_old\"\n" + - " },\n" + - " \"dest\":{\n" + - " \"index\":\"test_index_reindex\"\n" + - " }\n" + - "}"); - reindex.addParameter("wait_for_completion", "false"); - Map response = entityAsMap(client().performRequest(reindex)); - String taskId = (String) response.get("task"); - - // wait for task - Request getTask = new Request("GET", "/_tasks/" + taskId); - getTask.addParameter("wait_for_completion", "true"); - client().performRequest(getTask); - - // make sure .tasks index exists - assertBusy(() -> { - Request getTasksIndex = new Request("GET", "/.tasks"); - assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); - }); - } - - @SuppressWarnings("unchecked" + - "") - public void testSystemIndexGetsUpdatedMetadata() throws Exception { - assumeFalse("only run in upgraded cluster", isRunningAgainstOldCluster()); - - assertBusy(() -> { - Request clusterStateRequest = new Request("GET", "/_cluster/state/metadata"); - Map response = entityAsMap(client().performRequest(clusterStateRequest)); - Map metadata = (Map) response.get("metadata"); - assertNotNull(metadata); - Map indices = (Map) metadata.get("indices"); - assertNotNull(indices); - - Map tasksIndex = (Map) indices.get(".tasks"); - assertNotNull(tasksIndex); - assertThat(tasksIndex.get("system"), is(true)); - - Map testIndex = (Map) indices.get("test_index_old"); - assertNotNull(testIndex); - assertThat(testIndex.get("system"), is(false)); - }); + @SuppressWarnings("unchecked") + public void testSystemIndexMetadataIsUpgraded() throws Exception { + if (isRunningAgainstOldCluster()) { + // create index + Request createTestIndex = new Request("PUT", "/test_index_old"); + createTestIndex.setJsonEntity("{\"settings\": {\"index.number_of_replicas\": 0}}"); + client().performRequest(createTestIndex); + + Request bulk = new Request("POST", "/_bulk"); + bulk.addParameter("refresh", "true"); + bulk.setJsonEntity("{\"index\": {\"_index\": \"test_index_old\"}}\n" + + "{\"f1\": \"v1\", \"f2\": \"v2\"}\n"); + client().performRequest(bulk); + + // start a async reindex job + Request reindex = new Request("POST", "/_reindex"); + reindex.setJsonEntity( + "{\n" + + " \"source\":{\n" + + " \"index\":\"test_index_old\"\n" + + " },\n" + + " \"dest\":{\n" + + " \"index\":\"test_index_reindex\"\n" + + " }\n" + + "}"); + reindex.addParameter("wait_for_completion", "false"); + Map response = entityAsMap(client().performRequest(reindex)); + String taskId = (String) response.get("task"); + + // wait for task + Request getTask = new Request("GET", "/_tasks/" + taskId); + getTask.addParameter("wait_for_completion", "true"); + client().performRequest(getTask); + + // make sure .tasks index exists + assertBusy(() -> { + Request getTasksIndex = new Request("GET", "/.tasks"); + assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); + }); + } else { + assertBusy(() -> { + Request clusterStateRequest = new Request("GET", "/_cluster/state/metadata"); + Map response = entityAsMap(client().performRequest(clusterStateRequest)); + Map metadata = (Map) response.get("metadata"); + assertNotNull(metadata); + Map indices = (Map) metadata.get("indices"); + assertNotNull(indices); + + Map tasksIndex = (Map) indices.get(".tasks"); + assertNotNull(tasksIndex); + assertThat(tasksIndex.get("system"), is(true)); + + Map testIndex = (Map) indices.get("test_index_old"); + assertNotNull(testIndex); + assertThat(testIndex.get("system"), is(false)); + }); + } } public static void assertNumHits(String index, int numHits, int totalShards) throws IOException { diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java index 348c39b141ef9..9c590068192cf 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java @@ -27,69 +27,62 @@ public class SystemIndicesUpgradeIT extends AbstractRollingTestCase { - public void testOldDoesntHaveSystemIndexMetadata() throws Exception { - assumeTrue("only run in old cluster", CLUSTER_TYPE == ClusterType.OLD); - // create index - Request createTestIndex = new Request("PUT", "/test_index_old"); - createTestIndex.setJsonEntity("{\"settings\": {\"index.number_of_replicas\": 0}}"); - client().performRequest(createTestIndex); - - Request bulk = new Request("POST", "/_bulk"); - bulk.addParameter("refresh", "true"); - bulk.setJsonEntity("{\"index\": {\"_index\": \"test_index_old\"}}\n" + - "{\"f1\": \"v1\", \"f2\": \"v2\"}\n"); - client().performRequest(bulk); - - // start a async reindex job - Request reindex = new Request("POST", "/_reindex"); - reindex.setJsonEntity( - "{\n" + - " \"source\":{\n" + - " \"index\":\"test_index_old\"\n" + - " },\n" + - " \"dest\":{\n" + - " \"index\":\"test_index_reindex\"\n" + - " }\n" + - "}"); - reindex.addParameter("wait_for_completion", "false"); - Map response = entityAsMap(client().performRequest(reindex)); - String taskId = (String) response.get("task"); + @SuppressWarnings("unchecked") + public void testSystemIndicesUpgrades() throws Exception { + if (CLUSTER_TYPE == ClusterType.OLD) { + // create index + Request createTestIndex = new Request("PUT", "/test_index_old"); + createTestIndex.setJsonEntity("{\"settings\": {\"index.number_of_replicas\": 0}}"); + client().performRequest(createTestIndex); - // wait for task - Request getTask = new Request("GET", "/_tasks/" + taskId); - getTask.addParameter("wait_for_completion", "true"); - client().performRequest(getTask); + Request bulk = new Request("POST", "/_bulk"); + bulk.addParameter("refresh", "true"); + bulk.setJsonEntity("{\"index\": {\"_index\": \"test_index_old\"}}\n" + + "{\"f1\": \"v1\", \"f2\": \"v2\"}\n"); + client().performRequest(bulk); - // make sure .tasks index exists - assertBusy(() -> { - Request getTasksIndex = new Request("GET", "/.tasks"); - assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); - }); - } + // start a async reindex job + Request reindex = new Request("POST", "/_reindex"); + reindex.setJsonEntity( + "{\n" + + " \"source\":{\n" + + " \"index\":\"test_index_old\"\n" + + " },\n" + + " \"dest\":{\n" + + " \"index\":\"test_index_reindex\"\n" + + " }\n" + + "}"); + reindex.addParameter("wait_for_completion", "false"); + Map response = entityAsMap(client().performRequest(reindex)); + String taskId = (String) response.get("task"); - public void testMixedCluster() { - assumeTrue("nothing to do in mixed cluster", CLUSTER_TYPE == ClusterType.MIXED); - } - - @SuppressWarnings("unchecked") - public void testUpgradedCluster() throws Exception { - assumeTrue("only run on upgraded cluster", CLUSTER_TYPE == ClusterType.UPGRADED); + // wait for task + Request getTask = new Request("GET", "/_tasks/" + taskId); + getTask.addParameter("wait_for_completion", "true"); + client().performRequest(getTask); - assertBusy(() -> { - Request clusterStateRequest = new Request("GET", "/_cluster/state/metadata"); - Map response = entityAsMap(client().performRequest(clusterStateRequest)); - Map metadata = (Map) response.get("metadata"); - assertNotNull(metadata); - Map indices = (Map) metadata.get("indices"); - assertNotNull(indices); + // make sure .tasks index exists + assertBusy(() -> { + Request getTasksIndex = new Request("GET", "/.tasks"); + assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); + }); + } else if (CLUSTER_TYPE == ClusterType.UPGRADED) { + assertBusy(() -> { + Request clusterStateRequest = new Request("GET", "/_cluster/state/metadata"); + Map response = entityAsMap(client().performRequest(clusterStateRequest)); + Map metadata = (Map) response.get("metadata"); + assertNotNull(metadata); + Map indices = (Map) metadata.get("indices"); + assertNotNull(indices); - Map tasksIndex = (Map) indices.get(".tasks"); - assertNotNull(tasksIndex); - assertThat(tasksIndex.get("system"), is(true)); + Map tasksIndex = (Map) indices.get(".tasks"); + assertNotNull(tasksIndex); + assertThat(tasksIndex.get("system"), is(true)); - Map testIndex = (Map) indices.get("test_index_old"); - assertNotNull(testIndex); - assertThat(testIndex.get("system"), is(false)); - }); + Map testIndex = (Map) indices.get("test_index_old"); + assertNotNull(testIndex); + assertThat(testIndex.get("system"), is(false)); + }); + } } } From 1314cfd4404ade47614dd627868ee19d37c965dd Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 18:26:02 -0600 Subject: [PATCH 051/161] Add allow_system_index_access flag to docs as necessary & docs updates --- docs/reference/api-conventions.asciidoc | 12 ++ docs/reference/docs/delete-by-query.asciidoc | 76 ++++----- docs/reference/docs/update-by-query.asciidoc | 94 ++++++------ docs/reference/index-modules/blocks.asciidoc | 2 + docs/reference/indices/alias-exists.asciidoc | 2 + .../indices/apis/reload-analyzers.asciidoc | 2 + docs/reference/indices/clearcache.asciidoc | 2 + docs/reference/indices/close.asciidoc | 2 + docs/reference/indices/delete-index.asciidoc | 2 + docs/reference/indices/flush.asciidoc | 2 + docs/reference/indices/forcemerge.asciidoc | 2 + docs/reference/indices/get-alias.asciidoc | 2 + .../indices/get-field-mapping.asciidoc | 2 + docs/reference/indices/get-index.asciidoc | 2 + docs/reference/indices/get-mapping.asciidoc | 2 + docs/reference/indices/get-settings.asciidoc | 2 + .../reference/indices/indices-exists.asciidoc | 2 + docs/reference/indices/open-close.asciidoc | 2 + docs/reference/indices/put-mapping.asciidoc | 2 + docs/reference/indices/refresh.asciidoc | 4 +- docs/reference/indices/resolve.asciidoc | 2 + docs/reference/indices/segments.asciidoc | 6 +- docs/reference/indices/shard-stores.asciidoc | 2 + docs/reference/indices/stats.asciidoc | 4 +- .../indices/update-settings.asciidoc | 4 +- docs/reference/rest-api/common-parms.asciidoc | 14 +- docs/reference/search/count.asciidoc | 14 +- docs/reference/search/field-caps.asciidoc | 2 + docs/reference/search/multi-search.asciidoc | 4 + docs/reference/search/rank-eval.asciidoc | 144 +++++++++--------- docs/reference/search/search-shards.asciidoc | 4 +- .../reference/search/search-template.asciidoc | 40 ++--- docs/reference/search/search.asciidoc | 2 + docs/reference/search/validate.asciidoc | 22 +-- 34 files changed, 284 insertions(+), 198 deletions(-) diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index f458128ca661a..8e44cdfa61f71 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -37,6 +37,18 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] The defaults settings for the above parameters depend on the API being used. +Some indices (hereafter "system indices") are used by various system +components and/or plugins to store state or configuration. These indices +are not intended to be accessed directly, and accessing them directly is +deprecated. In the next major version, access to these indices will be prevented +by default to prevent accidental operations. You can opt in to access to these +indices by using the following query string parameter: + +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + +Using this parameter will prevent deprecation warnings in the current version, +and maintain the current behavior in the next major version. + Some multi-target APIs that can target indices also support the following query string parameter: diff --git a/docs/reference/docs/delete-by-query.asciidoc b/docs/reference/docs/delete-by-query.asciidoc index a65e75b1c37e7..55ebed7077ffb 100644 --- a/docs/reference/docs/delete-by-query.asciidoc +++ b/docs/reference/docs/delete-by-query.asciidoc @@ -53,13 +53,13 @@ POST /my-index-000001/_delete_by_query ==== {api-description-title} You can specify the query criteria in the request URI or the request body -using the same syntax as the <>. +using the same syntax as the <>. When you submit a delete by query request, {es} gets a snapshot of the data stream or index when it begins processing the request and deletes matching documents using `internal` versioning. If a document changes between the time that the snapshot is taken and the delete operation is processed, it results in a version -conflict and the delete operation fails. +conflict and the delete operation fails. NOTE: Documents with a version equal to 0 cannot be deleted using delete by query because `internal` versioning does not support 0 as a valid @@ -70,18 +70,18 @@ requests sequentially to find all of the matching documents to delete. A bulk delete request is performed for each batch of matching documents. If a search or bulk request is rejected, the requests are retried up to 10 times, with exponential back off. If the maximum retry limit is reached, processing halts -and all failed requests are returned in the response. Any delete requests that -completed successfully still stick, they are not rolled back. +and all failed requests are returned in the response. Any delete requests that +completed successfully still stick, they are not rolled back. -You can opt to count version conflicts instead of halting and returning by -setting `conflicts` to `proceed`. +You can opt to count version conflicts instead of halting and returning by +setting `conflicts` to `proceed`. ===== Refreshing shards Specifying the `refresh` parameter refreshes all shards involved in the delete -by query once the request completes. This is different than the delete API's -`refresh` parameter, which causes just the shard that received the delete -request to be refreshed. Unlike the delete API, it does not support +by query once the request completes. This is different than the delete API's +`refresh` parameter, which causes just the shard that received the delete +request to be refreshed. Unlike the delete API, it does not support `wait_for`. [[docs-delete-by-query-task-api]] @@ -90,7 +90,7 @@ request to be refreshed. Unlike the delete API, it does not support If the request contains `wait_for_completion=false`, {es} performs some preflight checks, launches the request, and returns a <> you can use to cancel or get the status of the task. {es} creates a -record of this task as a document at `.tasks/task/${taskId}`. When you are +record of this task as a document at `.tasks/task/${taskId}`. When you are done with a task, you should delete the task document so {es} can reclaim the space. @@ -101,20 +101,20 @@ before proceeding with the request. See <> for details. `timeout` controls how long each write request waits for unavailable shards to become available. Both work exactly the way they work in the <>. Delete by query uses scrolled searches, so you can also -specify the `scroll` parameter to control how long it keeps the search context +specify the `scroll` parameter to control how long it keeps the search context alive, for example `?scroll=10m`. The default is 5 minutes. ===== Throttling delete requests To control the rate at which delete by query issues batches of delete operations, you can set `requests_per_second` to any positive decimal number. This pads each -batch with a wait time to throttle the rate. Set `requests_per_second` to `-1` +batch with a wait time to throttle the rate. Set `requests_per_second` to `-1` to disable throttling. -Throttling uses a wait time between batches so that the internal scroll requests -can be given a timeout that takes the request padding into account. The padding -time is the difference between the batch size divided by the -`requests_per_second` and the time spent writing. By default the batch size is +Throttling uses a wait time between batches so that the internal scroll requests +can be given a timeout that takes the request padding into account. The padding +time is the difference between the batch size divided by the +`requests_per_second` and the time spent writing. By default the batch size is `1000`, so if `requests_per_second` is set to `500`: [source,txt] @@ -123,9 +123,9 @@ target_time = 1000 / 500 per second = 2 seconds wait_time = target_time - write_time = 2 seconds - .5 seconds = 1.5 seconds -------------------------------------------------- -Since the batch is issued as a single `_bulk` request, large batch sizes -cause {es} to create many requests and wait before starting the next set. -This is "bursty" instead of "smooth". +Since the batch is issued as a single `_bulk` request, large batch sizes +cause {es} to create many requests and wait before starting the next set. +This is "bursty" instead of "smooth". [[docs-delete-by-query-slice]] ===== Slicing @@ -134,11 +134,11 @@ Delete by query supports <> to parallelize the delete process. This can improve efficiency and provide a convenient way to break the request down into smaller parts. -Setting `slices` to `auto` chooses a reasonable number for most data streams and indices. -If you're slicing manually or otherwise tuning automatic slicing, keep in mind +Setting `slices` to `auto` chooses a reasonable number for most data streams and indices. +If you're slicing manually or otherwise tuning automatic slicing, keep in mind that: -* Query performance is most efficient when the number of `slices` is equal to +* Query performance is most efficient when the number of `slices` is equal to the number of shards in the index or backing index. If that number is large (for example, 500), choose a lower number as too many `slices` hurts performance. Setting `slices` higher than the number of shards generally does not improve efficiency @@ -171,25 +171,27 @@ Defaults to `true`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=analyzer] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=analyze_wildcard] - + `conflicts`:: - (Optional, string) What to do if delete by query hits version conflicts: + (Optional, string) What to do if delete by query hits version conflicts: `abort` or `proceed`. Defaults to `abort`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=default_operator] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=df] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=from] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=lenient] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=max_docs] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=preference] @@ -214,9 +216,9 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=scroll_size] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=search_type] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=search_timeout] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=slices] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=sort] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=source] @@ -226,7 +228,7 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=source_excludes] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=source_includes] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=stats] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=terminate_after] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=timeout] @@ -239,9 +241,9 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=wait_for_active_shards ==== {api-request-body-title} `query`:: - (Optional, <>) Specifies the documents to delete + (Optional, <>) Specifies the documents to delete using the <>. - + [[docs-delete-by-query-api-response-body]] ==== Response body @@ -345,7 +347,7 @@ this is non-empty then the request aborted because of those failures. Delete by query is implemented using batches, and any failure causes the entire process to abort but all failures in the current batch are collected into the array. You can use the `conflicts` option to prevent reindex from aborting on -version conflicts. +version conflicts. [[docs-delete-by-query-api-example]] ==== {api-examples-title} @@ -377,7 +379,7 @@ POST /my-index-000001,my-index-000002/_delete_by_query // TEST[s/^/PUT my-index-000001\nPUT my-index-000002\n/] Limit the delete by query operation to shards that a particular routing -value: +value: [source,console] -------------------------------------------------- @@ -571,7 +573,7 @@ though these are all taken at approximately the same time. The value of `requests_per_second` can be changed on a running delete by query using the `_rethrottle` API. Rethrottling that speeds up the -query takes effect immediately but rethrotting that slows down the query +query takes effect immediately but rethrotting that slows down the query takes effect after completing the current batch to prevent scroll timeouts. @@ -670,6 +672,6 @@ POST _tasks/r1A2WoRbTwKZ516z6NEs5A:36619/_cancel The task ID can be found using the <>. -Cancellation should happen quickly but might take a few seconds. The task status -API above will continue to list the delete by query task until this task checks that it +Cancellation should happen quickly but might take a few seconds. The task status +API above will continue to list the delete by query task until this task checks that it has been cancelled and terminates itself. diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index f7132c69a89d3..f75ebd56ac6f0 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -4,7 +4,7 @@ Update by query ++++ -Updates documents that match the specified query. +Updates documents that match the specified query. If no query is specified, performs an update on every document in the data stream or index without modifying the source, which is useful for picking up mapping changes. @@ -50,33 +50,33 @@ POST my-index-000001/_update_by_query?conflicts=proceed ==== {api-description-title} You can specify the query criteria in the request URI or the request body -using the same syntax as the <>. +using the same syntax as the <>. When you submit an update by query request, {es} gets a snapshot of the data stream or index when it begins processing the request and updates matching documents using -`internal` versioning. -When the versions match, the document is updated and the version number is incremented. -If a document changes between the time that the snapshot is taken and -the update operation is processed, it results in a version conflict and the operation fails. -You can opt to count version conflicts instead of halting and returning by -setting `conflicts` to `proceed`. +`internal` versioning. +When the versions match, the document is updated and the version number is incremented. +If a document changes between the time that the snapshot is taken and +the update operation is processed, it results in a version conflict and the operation fails. +You can opt to count version conflicts instead of halting and returning by +setting `conflicts` to `proceed`. NOTE: Documents with a version equal to 0 cannot be updated using update by query because `internal` versioning does not support 0 as a valid version number. While processing an update by query request, {es} performs multiple search -requests sequentially to find all of the matching documents. -A bulk update request is performed for each batch of matching documents. -Any query or update failures cause the update by query request to fail and +requests sequentially to find all of the matching documents. +A bulk update request is performed for each batch of matching documents. +Any query or update failures cause the update by query request to fail and the failures are shown in the response. Any update requests that completed successfully still stick, they are not rolled back. ===== Refreshing shards -Specifying the `refresh` parameter refreshes all shards once the request completes. +Specifying the `refresh` parameter refreshes all shards once the request completes. This is different than the update API's `refresh` parameter, which causes just the shard -that received the request to be refreshed. Unlike the update API, it does not support +that received the request to be refreshed. Unlike the update API, it does not support `wait_for`. [[docs-update-by-query-task-api]] @@ -84,9 +84,9 @@ that received the request to be refreshed. Unlike the update API, it does not su If the request contains `wait_for_completion=false`, {es} performs some preflight checks, launches the request, and returns a -<> you can use to cancel or get the status of the task. -{es} creates a record of this task as a document at `.tasks/task/${taskId}`. -When you are done with a task, you should delete the task document so +<> you can use to cancel or get the status of the task. +{es} creates a record of this task as a document at `.tasks/task/${taskId}`. +When you are done with a task, you should delete the task document so {es} can reclaim the space. ===== Waiting for active shards @@ -96,20 +96,20 @@ before proceeding with the request. See <> for details. `timeout` controls how long each write request waits for unavailable shards to become available. Both work exactly the way they work in the <>. Update by query uses scrolled searches, so you can also -specify the `scroll` parameter to control how long it keeps the search context +specify the `scroll` parameter to control how long it keeps the search context alive, for example `?scroll=10m`. The default is 5 minutes. ===== Throttling update requests To control the rate at which update by query issues batches of update operations, you can set `requests_per_second` to any positive decimal number. This pads each -batch with a wait time to throttle the rate. Set `requests_per_second` to `-1` +batch with a wait time to throttle the rate. Set `requests_per_second` to `-1` to disable throttling. -Throttling uses a wait time between batches so that the internal scroll requests -can be given a timeout that takes the request padding into account. The padding -time is the difference between the batch size divided by the -`requests_per_second` and the time spent writing. By default the batch size is +Throttling uses a wait time between batches so that the internal scroll requests +can be given a timeout that takes the request padding into account. The padding +time is the difference between the batch size divided by the +`requests_per_second` and the time spent writing. By default the batch size is `1000`, so if `requests_per_second` is set to `500`: [source,txt] @@ -118,9 +118,9 @@ target_time = 1000 / 500 per second = 2 seconds wait_time = target_time - write_time = 2 seconds - .5 seconds = 1.5 seconds -------------------------------------------------- -Since the batch is issued as a single `_bulk` request, large batch sizes -cause {es} to create many requests and wait before starting the next set. -This is "bursty" instead of "smooth". +Since the batch is issued as a single `_bulk` request, large batch sizes +cause {es} to create many requests and wait before starting the next set. +This is "bursty" instead of "smooth". [[docs-update-by-query-slice]] ===== Slicing @@ -129,11 +129,11 @@ Update by query supports <> to parallelize the update process. This can improve efficiency and provide a convenient way to break the request down into smaller parts. -Setting `slices` to `auto` chooses a reasonable number for most data streams and indices. -If you're slicing manually or otherwise tuning automatic slicing, keep in mind +Setting `slices` to `auto` chooses a reasonable number for most data streams and indices. +If you're slicing manually or otherwise tuning automatic slicing, keep in mind that: -* Query performance is most efficient when the number of `slices` is equal to +* Query performance is most efficient when the number of `slices` is equal to the number of shards in the index or backing index. If that number is large (for example, 500), choose a lower number as too many `slices` hurts performance. Setting `slices` higher than the number of shards generally does not improve efficiency @@ -166,25 +166,27 @@ Defaults to `true`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=analyzer] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=analyze_wildcard] - + `conflicts`:: - (Optional, string) What to do if delete by query hits version conflicts: + (Optional, string) What to do if delete by query hits version conflicts: `abort` or `proceed`. Defaults to `abort`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=default_operator] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=df] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=from] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=lenient] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=max_docs] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=pipeline] @@ -211,9 +213,9 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=scroll_size] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=search_type] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=search_timeout] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=slices] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=sort] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=source] @@ -223,7 +225,7 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=source_excludes] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=source_includes] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=stats] - + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=terminate_after] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=timeout] @@ -236,9 +238,9 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=wait_for_active_shards ==== {api-request-body-title} `query`:: - (Optional, <>) Specifies the documents to update + (Optional, <>) Specifies the documents to update using the <>. - + [[docs-update-by-query-api-response-body]] ==== Response body @@ -336,7 +338,7 @@ POST my-index-000001/_update_by_query?routing=1 -------------------------------------------------- // TEST[setup:my_index] -By default update by query uses scroll batches of 1000. +By default update by query uses scroll batches of 1000. You can change the batch size with the `scroll_size` parameter: [source,console] @@ -348,7 +350,7 @@ POST my-index-000001/_update_by_query?scroll_size=100 [[docs-update-by-query-api-source]] ===== Update the document source -Update by query supports scripts to update the document source. +Update by query supports scripts to update the document source. For example, the following request increments the `count` field for all documents with a `user.id` of `kimchy` in `my-index-000001`: @@ -390,16 +392,16 @@ operation that is performed: [horizontal] `noop`:: -Set `ctx.op = "noop"` if your script decides that it doesn't have to make any changes. +Set `ctx.op = "noop"` if your script decides that it doesn't have to make any changes. The update by query operation skips updating the document and increments the `noop` counter. `delete`:: -Set `ctx.op = "delete"` if your script decides that the document should be deleted. +Set `ctx.op = "delete"` if your script decides that the document should be deleted. The update by query operation deletes the document and increments the `deleted` counter. Update by query only supports `update`, `noop`, and `delete`. Setting `ctx.op` to anything else is an error. Setting any other field in `ctx` is an error. -This API only enables you to modify the source of matching documents, you cannot move them. +This API only enables you to modify the source of matching documents, you cannot move them. [[docs-update-by-query-api-ingest-pipeline]] ===== Update documents using an ingest pipeline @@ -485,7 +487,7 @@ of operations that the reindex expects to perform. You can estimate the progress by adding the `updated`, `created`, and `deleted` fields. The request will finish when their sum is equal to the `total` field. -With the task id you can look up the task directly. The following example +With the task id you can look up the task directly. The following example retrieves information about task `r1A2WoRbTwKZ516z6NEs5A:36619`: [source,console] @@ -515,8 +517,8 @@ POST _tasks/r1A2WoRbTwKZ516z6NEs5A:36619/_cancel The task ID can be found using the <>. -Cancellation should happen quickly but might take a few seconds. The task status -API above will continue to list the update by query task until this task checks +Cancellation should happen quickly but might take a few seconds. The task status +API above will continue to list the update by query task until this task checks that it has been cancelled and terminates itself. diff --git a/docs/reference/index-modules/blocks.asciidoc b/docs/reference/index-modules/blocks.asciidoc index d54b8f7e9be59..c35ed389abc16 100644 --- a/docs/reference/index-modules/blocks.asciidoc +++ b/docs/reference/index-modules/blocks.asciidoc @@ -129,6 +129,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=timeoutparms] diff --git a/docs/reference/indices/alias-exists.asciidoc b/docs/reference/indices/alias-exists.asciidoc index 2596fddef6b34..0139d78e0081e 100644 --- a/docs/reference/indices/alias-exists.asciidoc +++ b/docs/reference/indices/alias-exists.asciidoc @@ -40,6 +40,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `all`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=local] diff --git a/docs/reference/indices/apis/reload-analyzers.asciidoc b/docs/reference/indices/apis/reload-analyzers.asciidoc index eba55ff615bdc..029bd617eccff 100644 --- a/docs/reference/indices/apis/reload-analyzers.asciidoc +++ b/docs/reference/indices/apis/reload-analyzers.asciidoc @@ -85,6 +85,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/clearcache.asciidoc b/docs/reference/indices/clearcache.asciidoc index 7cfae6775ce16..342f5740c1fc3 100644 --- a/docs/reference/indices/clearcache.asciidoc +++ b/docs/reference/indices/clearcache.asciidoc @@ -45,6 +45,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + `fielddata`:: + -- diff --git a/docs/reference/indices/close.asciidoc b/docs/reference/indices/close.asciidoc index f008857d366d6..9a8b6ce95e42f 100644 --- a/docs/reference/indices/close.asciidoc +++ b/docs/reference/indices/close.asciidoc @@ -50,6 +50,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=wait_for_active_shards] diff --git a/docs/reference/indices/delete-index.asciidoc b/docs/reference/indices/delete-index.asciidoc index 19112da63d9f2..64d8dac0cf301 100644 --- a/docs/reference/indices/delete-index.asciidoc +++ b/docs/reference/indices/delete-index.asciidoc @@ -55,6 +55,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=timeoutparms] diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index 4f5d5e0638f73..e250d8a5cac04 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -72,6 +72,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + `force`:: + -- diff --git a/docs/reference/indices/forcemerge.asciidoc b/docs/reference/indices/forcemerge.asciidoc index 86ac22ba28494..5e34ef91ebab1 100644 --- a/docs/reference/indices/forcemerge.asciidoc +++ b/docs/reference/indices/forcemerge.asciidoc @@ -89,6 +89,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + `flush`:: (Optional, boolean) If `true`, diff --git a/docs/reference/indices/get-alias.asciidoc b/docs/reference/indices/get-alias.asciidoc index f2cb5ebab5ad4..8194744abf7f5 100644 --- a/docs/reference/indices/get-alias.asciidoc +++ b/docs/reference/indices/get-alias.asciidoc @@ -50,6 +50,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `all`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=local] diff --git a/docs/reference/indices/get-field-mapping.asciidoc b/docs/reference/indices/get-field-mapping.asciidoc index 9f7626703cbdc..83d3175935153 100644 --- a/docs/reference/indices/get-field-mapping.asciidoc +++ b/docs/reference/indices/get-field-mapping.asciidoc @@ -49,6 +49,8 @@ Defaults to `true`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] `include_defaults`:: diff --git a/docs/reference/indices/get-index.asciidoc b/docs/reference/indices/get-index.asciidoc index 3087790b6ecc2..dcf7481bd4240 100644 --- a/docs/reference/indices/get-index.asciidoc +++ b/docs/reference/indices/get-index.asciidoc @@ -42,6 +42,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=include-defaults] diff --git a/docs/reference/indices/get-mapping.asciidoc b/docs/reference/indices/get-mapping.asciidoc index df71e073704ae..7d37029235063 100644 --- a/docs/reference/indices/get-mapping.asciidoc +++ b/docs/reference/indices/get-mapping.asciidoc @@ -44,6 +44,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=local] diff --git a/docs/reference/indices/get-settings.asciidoc b/docs/reference/indices/get-settings.asciidoc index f655a7ff18a01..fe12cee8e8dc0 100644 --- a/docs/reference/indices/get-settings.asciidoc +++ b/docs/reference/indices/get-settings.asciidoc @@ -49,6 +49,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `all`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=include-defaults] diff --git a/docs/reference/indices/indices-exists.asciidoc b/docs/reference/indices/indices-exists.asciidoc index 6b9e9789bb38a..264c92cf8336a 100644 --- a/docs/reference/indices/indices-exists.asciidoc +++ b/docs/reference/indices/indices-exists.asciidoc @@ -41,6 +41,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=include-defaults] diff --git a/docs/reference/indices/open-close.asciidoc b/docs/reference/indices/open-close.asciidoc index 27036cb1129b9..dc60cee63ed23 100644 --- a/docs/reference/indices/open-close.asciidoc +++ b/docs/reference/indices/open-close.asciidoc @@ -106,6 +106,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `closed`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=wait_for_active_shards] diff --git a/docs/reference/indices/put-mapping.asciidoc b/docs/reference/indices/put-mapping.asciidoc index 0ea7fdadcbe39..9d06ce8c87010 100644 --- a/docs/reference/indices/put-mapping.asciidoc +++ b/docs/reference/indices/put-mapping.asciidoc @@ -53,6 +53,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=timeoutparms] diff --git a/docs/reference/indices/refresh.asciidoc b/docs/reference/indices/refresh.asciidoc index f8259b9d5ba32..109799f48347a 100644 --- a/docs/reference/indices/refresh.asciidoc +++ b/docs/reference/indices/refresh.asciidoc @@ -49,7 +49,7 @@ refresh operation completes. ==== Refreshes are resource-intensive. To ensure good cluster performance, -we recommend waiting for {es}'s periodic refresh +we recommend waiting for {es}'s periodic refresh rather than performing an explicit refresh when possible. @@ -84,6 +84,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/resolve.asciidoc b/docs/reference/indices/resolve.asciidoc index 3f0a5978f114b..f38fd3fd00100 100644 --- a/docs/reference/indices/resolve.asciidoc +++ b/docs/reference/indices/resolve.asciidoc @@ -76,6 +76,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + [[resolve-index-api-example]] ==== {api-examples-title} diff --git a/docs/reference/indices/segments.asciidoc b/docs/reference/indices/segments.asciidoc index 3aba8683cce03..76cdd9debddae 100644 --- a/docs/reference/indices/segments.asciidoc +++ b/docs/reference/indices/segments.asciidoc @@ -46,6 +46,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] `verbose`:: @@ -60,7 +62,7 @@ Defaults to `false`. ==== {api-response-body-title} ``:: -(String) +(String) include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=segment] `generation`:: @@ -83,7 +85,7 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=segment-size] (Integer) include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=memory] -`committed`:: +`committed`:: (Boolean) include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=committed] diff --git a/docs/reference/indices/shard-stores.asciidoc b/docs/reference/indices/shard-stores.asciidoc index 88e8f248a5317..f0c6717225d65 100644 --- a/docs/reference/indices/shard-stores.asciidoc +++ b/docs/reference/indices/shard-stores.asciidoc @@ -64,6 +64,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] `status`:: diff --git a/docs/reference/indices/stats.asciidoc b/docs/reference/indices/stats.asciidoc index 0b89de23c9830..99eff6e0a6540 100644 --- a/docs/reference/indices/stats.asciidoc +++ b/docs/reference/indices/stats.asciidoc @@ -33,7 +33,7 @@ more data streams and indices. By default, the returned statistics are index-level with `primaries` and `total` aggregations. -`primaries` are the values for only the primary shards. +`primaries` are the values for only the primary shards. `total` are the accumulated values for both primary and replica shards. To get shard-level statistics, @@ -71,6 +71,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=fields] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=completion-fields] diff --git a/docs/reference/indices/update-settings.asciidoc b/docs/reference/indices/update-settings.asciidoc index ea42112980322..60bce615e6abe 100644 --- a/docs/reference/indices/update-settings.asciidoc +++ b/docs/reference/indices/update-settings.asciidoc @@ -50,6 +50,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] @@ -147,7 +149,7 @@ and reopen the index. [NOTE] ==== -You cannot close the write index of a data stream. +You cannot close the write index of a data stream. To update the analyzer for a data stream's write index and future backing indices, update the analyzer in the <> -IMPORTANT: {transforms-cap} support a subset of the functionality in +IMPORTANT: {transforms-cap} support a subset of the functionality in aggregations. See <>. -- @@ -685,7 +693,7 @@ Defines how to group the data. More than one grouping can be defined + -- * <<_date_histogram,Date histogram>> -* <<_geotile_grid,Geotile Grid>> +* <<_geotile_grid,Geotile Grid>> * <<_histogram,Histogram>> * <<_terms,Terms>> diff --git a/docs/reference/search/count.asciidoc b/docs/reference/search/count.asciidoc index 5bd58cbf9b94c..57e9ffbeb7662 100644 --- a/docs/reference/search/count.asciidoc +++ b/docs/reference/search/count.asciidoc @@ -22,16 +22,16 @@ the <> works. [[search-count-api-desc]] ==== {api-description-title} -The count API allows you to execute a query and get the number of matches for -that query. The query can either -be provided using a simple query string as a parameter, or using the +The count API allows you to execute a query and get the number of matches for +that query. The query can either +be provided using a simple query string as a parameter, or using the <> defined within the request body. The count API supports <>. You can run a single count API search across multiple data streams and indices. -The operation is broadcast across all shards. For each shard id group, a replica -is chosen and executed against it. This means that replicas increase the +The operation is broadcast across all shards. For each shard id group, a replica +is chosen and executed against it. This means that replicas increase the scalability of count. @@ -66,6 +66,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=ignore_throttled] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] @@ -74,7 +76,7 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=lenient] `min_score`:: (Optional, float) - Sets the minimum `_score` value that documents must have to be included in the + Sets the minimum `_score` value that documents must have to be included in the result. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=preference] diff --git a/docs/reference/search/field-caps.asciidoc b/docs/reference/search/field-caps.asciidoc index a4bc8eae30c5c..a7692aaf93a23 100644 --- a/docs/reference/search/field-caps.asciidoc +++ b/docs/reference/search/field-caps.asciidoc @@ -56,6 +56,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. -- +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=fields] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/search/multi-search.asciidoc b/docs/reference/search/multi-search.asciidoc index 3642c09967cdd..5467d29bc0f84 100644 --- a/docs/reference/search/multi-search.asciidoc +++ b/docs/reference/search/multi-search.asciidoc @@ -78,6 +78,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + `ignore_throttled`:: (Optional, boolean) If `true`, concrete, expanded or aliased indices are ignored when frozen. @@ -183,6 +185,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + `ignore_unavailable`::: (Optional, boolean) If `true`, documents from missing or closed indices are not included in the response. Defaults to `false`. diff --git a/docs/reference/search/rank-eval.asciidoc b/docs/reference/search/rank-eval.asciidoc index 1970c2c568a73..c90644c81b478 100644 --- a/docs/reference/search/rank-eval.asciidoc +++ b/docs/reference/search/rank-eval.asciidoc @@ -4,7 +4,7 @@ Ranking evaluation ++++ -Allows you to evaluate the quality of ranked search results over a set of +Allows you to evaluate the quality of ranked search results over a set of typical search queries. [[search-rank-eval-api-request]] @@ -18,46 +18,46 @@ typical search queries. [[search-rank-eval-api-desc]] ==== {api-description-title} -The ranking evaluation API allows you to evaluate the quality of ranked search +The ranking evaluation API allows you to evaluate the quality of ranked search results over a set of typical search queries. Given this set of queries and a list of manually rated documents, the `_rank_eval` endpoint calculates and returns typical information retrieval metrics like _mean reciprocal rank_, _precision_ or _discounted cumulative gain_. -Search quality evaluation starts with looking at the users of your search -application, and the things that they are searching for. Users have a specific -_information need_; for example, they are looking for gift in a web shop or want -to book a flight for their next holiday. They usually enter some search terms -into a search box or some other web form. All of this information, together with -meta information about the user (for example the browser, location, earlier -preferences and so on) then gets translated into a query to the underlying +Search quality evaluation starts with looking at the users of your search +application, and the things that they are searching for. Users have a specific +_information need_; for example, they are looking for gift in a web shop or want +to book a flight for their next holiday. They usually enter some search terms +into a search box or some other web form. All of this information, together with +meta information about the user (for example the browser, location, earlier +preferences and so on) then gets translated into a query to the underlying search system. -The challenge for search engineers is to tweak this translation process from -user entries to a concrete query, in such a way that the search results contain -the most relevant information with respect to the user's information need. This -can only be done if the search result quality is evaluated constantly across a -representative test suite of typical user queries, so that improvements in the -rankings for one particular query don't negatively affect the ranking for +The challenge for search engineers is to tweak this translation process from +user entries to a concrete query, in such a way that the search results contain +the most relevant information with respect to the user's information need. This +can only be done if the search result quality is evaluated constantly across a +representative test suite of typical user queries, so that improvements in the +rankings for one particular query don't negatively affect the ranking for other types of queries. In order to get started with search quality evaluation, you need three basic things: -. A collection of documents you want to evaluate your query performance against, +. A collection of documents you want to evaluate your query performance against, usually one or more data streams or indices. . A collection of typical search requests that users enter into your system. . A set of document ratings that represent the documents' relevance with respect to a search request. - -It is important to note that one set of document ratings is needed per test -query, and that the relevance judgements are based on the information need of + +It is important to note that one set of document ratings is needed per test +query, and that the relevance judgements are based on the information need of the user that entered the query. -The ranking evaluation API provides a convenient way to use this information in -a ranking evaluation request to calculate different search evaluation metrics. -This gives you a first estimation of your overall search quality, as well as a -measurement to optimize against when fine-tuning various aspect of the query +The ranking evaluation API provides a convenient way to use this information in +a ranking evaluation request to calculate different search evaluation metrics. +This gives you a first estimation of your overall search quality, as well as a +measurement to optimize against when fine-tuning various aspect of the query generation in your application. @@ -85,6 +85,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. -- +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] @@ -97,7 +99,7 @@ In its most basic form, a request to the `_rank_eval` endpoint has two sections: ----------------------------- GET /my-index-000001/_rank_eval { - "requests": [ ... ], <1> + "requests": [ ... ], <1> "metric": { <2> "mean_reciprocal_rank": { ... } <3> } @@ -109,7 +111,7 @@ GET /my-index-000001/_rank_eval <2> definition of the evaluation metric to calculate <3> a specific metric and its parameters -The request section contains several search requests typical to your +The request section contains several search requests typical to your application, along with the document ratings for each particular search request. [source,js] @@ -122,7 +124,7 @@ GET /my-index-000001/_rank_eval "request": { <2> "query": { "match": { "text": "amsterdam" } } }, - "ratings": [ <3> + "ratings": [ <3> { "_index": "my-index-000001", "_id": "doc1", "rating": 0 }, { "_index": "my-index-000001", "_id": "doc2", "rating": 3 }, { "_index": "my-index-000001", "_id": "doc3", "rating": 1 } @@ -150,38 +152,38 @@ GET /my-index-000001/_rank_eval - `_id`: The document ID. - `rating`: The document's relevance with regard to this search request. -A document `rating` can be any integer value that expresses the relevance of the -document on a user-defined scale. For some of the metrics, just giving a binary -rating (for example `0` for irrelevant and `1` for relevant) will be sufficient, +A document `rating` can be any integer value that expresses the relevance of the +document on a user-defined scale. For some of the metrics, just giving a binary +rating (for example `0` for irrelevant and `1` for relevant) will be sufficient, while other metrics can use a more fine-grained scale. ===== Template-based ranking evaluation -As an alternative to having to provide a single query per test request, it is -possible to specify query templates in the evaluation request and later refer to -them. This way, queries with a similar structure that differ only in their -parameters don't have to be repeated all the time in the `requests` section. -In typical search systems, where user inputs usually get filled into a small +As an alternative to having to provide a single query per test request, it is +possible to specify query templates in the evaluation request and later refer to +them. This way, queries with a similar structure that differ only in their +parameters don't have to be repeated all the time in the `requests` section. +In typical search systems, where user inputs usually get filled into a small set of query templates, this helps make the evaluation request more succinct. [source,js] -------------------------------- GET /my-index-000001/_rank_eval -{ +{ [...] "templates": [ { "id": "match_one_field_query", <1> "template": { <2> - "inline": { - "query": { + "inline": { + "query": { "match": { "{{field}}": { "query": "{{query_string}}" }} } } } } - ], + ], "requests": [ { "id": "amsterdam_query", @@ -197,7 +199,7 @@ GET /my-index-000001/_rank_eval -------------------------------- // NOTCONSOLE -<1> the template id +<1> the template id <2> the template definition to use <3> a reference to a previously defined template <4> the parameters to use to fill the template @@ -205,7 +207,7 @@ GET /my-index-000001/_rank_eval ===== Available evaluation metrics -The `metric` section determines which of the available evaluation metrics +The `metric` section determines which of the available evaluation metrics will be used. The following metrics are supported: [discrete] @@ -254,8 +256,8 @@ The `precision` metric takes the following optional parameters [cols="<,<",options="header",] |======================================================================= |Parameter |Description -|`k` |sets the maximum number of documents retrieved per query. This value will act in place of the usual `size` parameter -in the query. Defaults to 10. +|`k` |sets the maximum number of documents retrieved per query. This value will act in place of the usual `size` parameter +in the query. Defaults to 10. |`relevant_rating_threshold` |sets the rating threshold above which documents are considered to be "relevant". Defaults to `1`. |`ignore_unlabeled` |controls how unlabeled documents in the search results are counted. @@ -318,10 +320,10 @@ in the query. Defaults to 10. [discrete] ===== Mean reciprocal rank -For every query in the test suite, this metric calculates the reciprocal of the -rank of the first relevant document. For example, finding the first relevant -result in position 3 means the reciprocal rank is 1/3. The reciprocal rank for -each query is averaged across all queries in the test suite to give the +For every query in the test suite, this metric calculates the reciprocal of the +rank of the first relevant document. For example, finding the first relevant +result in position 3 means the reciprocal rank is 1/3. The reciprocal rank for +each query is averaged across all queries in the test suite to give the {wikipedia}/Mean_reciprocal_rank[mean reciprocal rank]. [source,console] @@ -349,7 +351,7 @@ The `mean_reciprocal_rank` metric takes the following optional parameters [cols="<,<",options="header",] |======================================================================= |Parameter |Description -|`k` |sets the maximum number of documents retrieved per query. This value will act in place of the usual `size` parameter +|`k` |sets the maximum number of documents retrieved per query. This value will act in place of the usual `size` parameter in the query. Defaults to 10. |`relevant_rating_threshold` |Sets the rating threshold above which documents are considered to be "relevant". Defaults to `1`. @@ -359,13 +361,13 @@ in the query. Defaults to 10. [discrete] ===== Discounted cumulative gain (DCG) -In contrast to the two metrics above, -{wikipedia}/Discounted_cumulative_gain[discounted cumulative gain] +In contrast to the two metrics above, +{wikipedia}/Discounted_cumulative_gain[discounted cumulative gain] takes both the rank and the rating of the search results into account. -The assumption is that highly relevant documents are more useful for the user -when appearing at the top of the result list. Therefore, the DCG formula reduces -the contribution that high ratings for documents on lower search ranks have on +The assumption is that highly relevant documents are more useful for the user +when appearing at the top of the result list. Therefore, the DCG formula reduces +the contribution that high ratings for documents on lower search ranks have on the overall DCG metric. [source,console] @@ -393,7 +395,7 @@ The `dcg` metric takes the following optional parameters: [cols="<,<",options="header",] |======================================================================= |Parameter |Description -|`k` |sets the maximum number of documents retrieved per query. This value will act in place of the usual `size` parameter +|`k` |sets the maximum number of documents retrieved per query. This value will act in place of the usual `size` parameter in the query. Defaults to 10. |`normalize` | If set to `true`, this metric will calculate the {wikipedia}/Discounted_cumulative_gain#Normalized_DCG[Normalized DCG]. |======================================================================= @@ -402,26 +404,26 @@ in the query. Defaults to 10. [discrete] ===== Expected Reciprocal Rank (ERR) -Expected Reciprocal Rank (ERR) is an extension of the classical reciprocal rank -for the graded relevance case (Olivier Chapelle, Donald Metzler, Ya Zhang, and -Pierre Grinspan. 2009. +Expected Reciprocal Rank (ERR) is an extension of the classical reciprocal rank +for the graded relevance case (Olivier Chapelle, Donald Metzler, Ya Zhang, and +Pierre Grinspan. 2009. https://olivier.chapelle.cc/pub/err.pdf[Expected reciprocal rank for graded relevance].) -It is based on the assumption of a cascade model of search, in which a user -scans through ranked search results in order and stops at the first document -that satisfies the information need. For this reason, it is a good metric for -question answering and navigation queries, but less so for survey-oriented -information needs where the user is interested in finding many relevant +It is based on the assumption of a cascade model of search, in which a user +scans through ranked search results in order and stops at the first document +that satisfies the information need. For this reason, it is a good metric for +question answering and navigation queries, but less so for survey-oriented +information needs where the user is interested in finding many relevant documents in the top k results. -The metric models the expectation of the reciprocal of the position at which a +The metric models the expectation of the reciprocal of the position at which a user stops reading through the result list. This means that a relevant document -in a top ranking position will have a large contribution to the overall score. -However, the same document will contribute much less to the score if it appears -in a lower rank; even more so if there are some relevant (but maybe less relevant) -documents preceding it. In this way, the ERR metric discounts documents that -are shown after very relevant documents. This introduces a notion of dependency -in the ordering of relevant documents that e.g. Precision or DCG don't account +in a top ranking position will have a large contribution to the overall score. +However, the same document will contribute much less to the score if it appears +in a lower rank; even more so if there are some relevant (but maybe less relevant) +documents preceding it. In this way, the ERR metric discounts documents that +are shown after very relevant documents. This introduces a notion of dependency +in the ordering of relevant documents that e.g. Precision or DCG don't account for. [source,console] @@ -458,9 +460,9 @@ in the query. Defaults to 10. ===== Response format -The response of the `_rank_eval` endpoint contains the overall calculated result -for the defined quality metric, a `details` section with a breakdown of results -for each query in the test suite and an optional `failures` section that shows +The response of the `_rank_eval` endpoint contains the overall calculated result +for the defined quality metric, a `details` section with a breakdown of results +for each query in the test suite and an optional `failures` section that shows potential errors of individual queries. The response has the following format: [source,js] diff --git a/docs/reference/search/search-shards.asciidoc b/docs/reference/search/search-shards.asciidoc index 5fee6f24fc1df..6cf6843df2728 100644 --- a/docs/reference/search/search-shards.asciidoc +++ b/docs/reference/search/search-shards.asciidoc @@ -44,6 +44,8 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. -- +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=local] @@ -186,5 +188,5 @@ The API returns the following result: // TESTRESPONSE[s/0TvkCyF7TAmM1wHP4a42-A/$body.shards.1.0.allocation_id.id/] // TESTRESPONSE[s/fMju3hd1QHWmWrIgFnI4Ww/$body.shards.0.0.allocation_id.id/] -Because of the specified routing values, +Because of the specified routing values, the search is only executed against two of the shards. diff --git a/docs/reference/search/search-template.asciidoc b/docs/reference/search/search-template.asciidoc index 63f9b2650396a..bf56509c957b0 100644 --- a/docs/reference/search/search-template.asciidoc +++ b/docs/reference/search/search-template.asciidoc @@ -29,17 +29,17 @@ GET _search/template [[search-template-api-desc]] ==== {api-description-title} -The `/_search/template` endpoint allows you to use the mustache language to pre- -render search requests, before they are executed and fill existing templates +The `/_search/template` endpoint allows you to use the mustache language to pre- +render search requests, before they are executed and fill existing templates with template parameters. For more information on how Mustache templating and what kind of templating you can do with it check out the https://mustache.github.io/mustache.5.html[online documentation of the mustache project]. -NOTE: The mustache language is implemented in {es} as a sandboxed scripting -language, hence it obeys settings that may be used to enable or disable scripts -per type and context as described in the +NOTE: The mustache language is implemented in {es} as a sandboxed scripting +language, hence it obeys settings that may be used to enable or disable scripts +per type and context as described in the <>. @@ -57,17 +57,19 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-no-indices] Defaults to `true`. `ccs_minimize_roundtrips`:: - (Optional, boolean) If `true`, network round-trips are minimized for + (Optional, boolean) If `true`, network round-trips are minimized for cross-cluster search requests. Defaults to `true`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] +include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + `explain`:: - (Optional, boolean) If `true`, the response includes additional details about + (Optional, boolean) If `true`, the response includes additional details about score computation as part of a hit. Defaults to `false`. `ignore_throttled`:: - (Optional, boolean) If `true`, specified concrete, expanded or aliased indices + (Optional, boolean) If `true`, specified concrete, expanded or aliased indices are not included in the response when throttled. Defaults to `true`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] @@ -75,11 +77,11 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailab include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=preference] `profile`:: - (Optional, boolean) If `true`, the query execution is profiled. Defaults + (Optional, boolean) If `true`, the query execution is profiled. Defaults to `false`. `rest_total_hits_as_int`:: - (Optional, boolean) If `true`, `hits.total` are rendered as an integer in + (Optional, boolean) If `true`, `hits.total` are rendered as an integer in the response. Defaults to `false`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=routing] @@ -89,9 +91,9 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=scroll] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=search_type] `typed_keys`:: - (Optional, boolean) If `true`, aggregation and suggester names are + (Optional, boolean) If `true`, aggregation and suggester names are prefixed by their respective types in the response. Defaults to `false`. - + [[search-template-api-request-body]] ==== {api-request-body-title} @@ -128,7 +130,7 @@ POST _scripts/ ////////////////////////// -The API returns the following result if the template has been successfully +The API returns the following result if the template has been successfully created: [source,console-result] @@ -198,7 +200,7 @@ GET _search/template [[_validating_templates]] ==== Validating a search template -A template can be rendered in a response with given parameters by using the +A template can be rendered in a response with given parameters by using the following request: [source,console] @@ -603,7 +605,7 @@ query as a string instead: ===== Encoding URLs The `{{#url}}value{{/url}}` function can be used to encode a string value -in a HTML encoding form as defined in by the +in a HTML encoding form as defined in by the https://www.w3.org/TR/html4/[HTML specification]. As an example, it is useful to encode a URL: @@ -657,7 +659,7 @@ Allows to execute several search template requests. [[multi-search-template-api-desc]] ==== {api-description-title} -Allows to execute several search template requests within the same API using the +Allows to execute several search template requests within the same API using the `_msearch/template` endpoint. The format of the request is similar to the <> works -If the query is invalid, `valid` will be `false`. Here the query is invalid -because {es} knows the `post_date` field should be a date due to dynamic +If the query is invalid, `valid` will be `false`. Here the query is invalid +because {es} knows the `post_date` field should be a date due to dynamic mapping, and 'foo' does not correctly parse into a date: [source,console] @@ -154,7 +156,7 @@ GET my-index-000001/_validate/query ===== The explain parameter -An `explain` parameter can be specified to get more detailed information about +An `explain` parameter can be specified to get more detailed information about why a query failed: [source,console] @@ -194,8 +196,8 @@ The API returns the following response: ===== The rewrite parameter -When the query is valid, the explanation defaults to the string representation -of that query. With `rewrite` set to `true`, the explanation is more detailed +When the query is valid, the explanation defaults to the string representation +of that query. With `rewrite` set to `true`, the explanation is more detailed showing the actual Lucene query that will be executed. [source,console] From f0a9b68fa3834f0a4bcd6046d5989b75d4bb1ed6 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 18:49:28 -0600 Subject: [PATCH 052/161] Add allow_no_indices to upgrade tests to ensure they fail if appropriate --- .../java/org/elasticsearch/upgrades/FullClusterRestartIT.java | 1 + .../java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java | 1 + 2 files changed, 2 insertions(+) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index cdbda0564bf0c..57166309d95c2 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -1422,6 +1422,7 @@ public void testSystemIndexMetadataIsUpgraded() throws Exception { // make sure .tasks index exists assertBusy(() -> { Request getTasksIndex = new Request("GET", "/.tasks"); + getTasksIndex.addParameter("allow_no_indices", "false"); assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); }); } else { diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java index 9c590068192cf..dbfed10453fad 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java @@ -64,6 +64,7 @@ public void testSystemIndicesUpgrades() throws Exception { // make sure .tasks index exists assertBusy(() -> { Request getTasksIndex = new Request("GET", "/.tasks"); + getTasksIndex.addParameter("allow_no_indices", "false"); assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); }); } else if (CLUSTER_TYPE == ClusterType.UPGRADED) { From c2f21d2954f25afeaf399fa2bfb729a6a1b5c8c6 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 18:55:41 -0600 Subject: [PATCH 053/161] Ensure we throw a 404 if a system index is explicitly requested, but not allowed --- .../cluster/metadata/IndexNameExpressionResolver.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 3c91981a40429..4465f4da51e22 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -851,6 +851,8 @@ private Set innerResolve(Context context, List expressions, Indi } else if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM && context.includeDataStreams() == false) { throw indexNotFoundException(expression); + } else if (indexAbstraction.isSystem() && context.isSystemIndexAccessAllowed() == false) { + throw indexNotFoundException(expression); } } if (add) { From a17155c1de71005e39be572c36816a6a63fbe047 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 17 Aug 2020 19:23:35 -0600 Subject: [PATCH 054/161] Upgrade tests for alias logic --- .../upgrades/FullClusterRestartIT.java | 17 +++++++++++++++++ .../upgrades/SystemIndicesUpgradeIT.java | 18 ++++++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index 57166309d95c2..743c139c49499 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -62,6 +62,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -1425,6 +1426,15 @@ public void testSystemIndexMetadataIsUpgraded() throws Exception { getTasksIndex.addParameter("allow_no_indices", "false"); assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); }); + // Create an alias to make sure it gets upgraded properly + Request putAliasRequest = new Request("POST", "/_aliases"); + putAliasRequest.setJsonEntity("{\n" + + " \"actions\": [\n" + + " {\"add\": {\"index\": \".tasks\", \"alias\": \"test-system-alias\"}},\n" + + " {\"add\": {\"index\": \"test_index_reindex\", \"alias\": \"test-system-alias\"}}\n" + + " ]\n" + + "}"); + assertThat(client().performRequest(putAliasRequest).getStatusLine().getStatusCode(), is(200)); } else { assertBusy(() -> { Request clusterStateRequest = new Request("GET", "/_cluster/state/metadata"); @@ -1442,6 +1452,13 @@ public void testSystemIndexMetadataIsUpgraded() throws Exception { assertNotNull(testIndex); assertThat(testIndex.get("system"), is(false)); }); + + // Verify that the alias survived the upgrade + Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); + getAliasRequest.addParameter("allow_system_index_access", "true"); + Map response = entityAsMap(client().performRequest(getAliasRequest)); + assertThat(response, hasKey(".tasks")); + assertThat(response, hasKey("test_index_reindex")); } } diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java index dbfed10453fad..26a0813a134eb 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java @@ -23,6 +23,7 @@ import java.util.Map; +import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.is; public class SystemIndicesUpgradeIT extends AbstractRollingTestCase { @@ -67,6 +68,16 @@ public void testSystemIndicesUpgrades() throws Exception { getTasksIndex.addParameter("allow_no_indices", "false"); assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); }); + + // Create an alias to make sure it gets upgraded properly + Request putAliasRequest = new Request("POST", "/_aliases"); + putAliasRequest.setJsonEntity("{\n" + + " \"actions\": [\n" + + " {\"add\": {\"index\": \".tasks\", \"alias\": \"test-system-alias\"}},\n" + + " {\"add\": {\"index\": \"test_index_reindex\", \"alias\": \"test-system-alias\"}}\n" + + " ]\n" + + "}"); + assertThat(client().performRequest(putAliasRequest).getStatusLine().getStatusCode(), is(200)); } else if (CLUSTER_TYPE == ClusterType.UPGRADED) { assertBusy(() -> { Request clusterStateRequest = new Request("GET", "/_cluster/state/metadata"); @@ -84,6 +95,13 @@ public void testSystemIndicesUpgrades() throws Exception { assertNotNull(testIndex); assertThat(testIndex.get("system"), is(false)); }); + + // Verify that the alias survived the upgrade + Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); + getAliasRequest.addParameter("allow_system_index_access", "true"); + Map response = entityAsMap(client().performRequest(getAliasRequest)); + assertThat(response, hasKey(".tasks")); + assertThat(response, hasKey("test_index_reindex")); } } } From 8d4f90c9c6f8dc0a35b2bbfbc6dd6f348a7cf1da Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 18 Aug 2020 14:59:18 -0600 Subject: [PATCH 055/161] Fix BWC tests to account for both upgrades from 7.x and current --- .../upgrades/FullClusterRestartIT.java | 82 ++++++++++++------- .../upgrades/SystemIndicesUpgradeIT.java | 81 +++++++++++------- 2 files changed, 104 insertions(+), 59 deletions(-) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index 743c139c49499..6f114abbe31ac 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -23,6 +23,7 @@ import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.MetadataIndexStateService; @@ -35,6 +36,7 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.test.NotEqualMessageBuilder; +import org.elasticsearch.test.XContentTestUtils; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ObjectPath; import org.junit.Before; @@ -55,9 +57,11 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.singletonList; import static java.util.Collections.singletonMap; +import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ENFORCEMENT_VERSION; import static org.elasticsearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.rest.BaseRestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -1421,44 +1425,62 @@ public void testSystemIndexMetadataIsUpgraded() throws Exception { client().performRequest(getTask); // make sure .tasks index exists + Request getTasksIndex = new Request("GET", "/.tasks"); + getTasksIndex.addParameter("allow_no_indices", "false"); + + // We run "upgrade" tests from the current version as well, so if we're on a recent enough version we need to specify that + // we need system index access here. + if (minimumNodeVersion().onOrAfter(ALLOW_SYSTEM_INDEX_ADDED_VERSION)) { + getTasksIndex.addParameter("allow_system_index_access", "true"); + } assertBusy(() -> { - Request getTasksIndex = new Request("GET", "/.tasks"); - getTasksIndex.addParameter("allow_no_indices", "false"); - assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); + try { + assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); + } catch (ResponseException e) { + throw new AssertionError(".tasks index does not exist yet"); + } }); - // Create an alias to make sure it gets upgraded properly - Request putAliasRequest = new Request("POST", "/_aliases"); - putAliasRequest.setJsonEntity("{\n" + - " \"actions\": [\n" + - " {\"add\": {\"index\": \".tasks\", \"alias\": \"test-system-alias\"}},\n" + - " {\"add\": {\"index\": \"test_index_reindex\", \"alias\": \"test-system-alias\"}}\n" + - " ]\n" + - "}"); - assertThat(client().performRequest(putAliasRequest).getStatusLine().getStatusCode(), is(200)); + + // If we are on 7.x create an alias that includes both a system index and a non-system index so we can be sure it gets + // upgraded properly. If we're already on 8.x, skip this part of the test. + if (minimumNodeVersion().before(SYSTEM_INDEX_ENFORCEMENT_VERSION)) { + // Create an alias to make sure it gets upgraded properly + Request putAliasRequest = new Request("POST", "/_aliases"); + putAliasRequest.setJsonEntity("{\n" + + " \"actions\": [\n" + + " {\"add\": {\"index\": \".tasks\", \"alias\": \"test-system-alias\"}},\n" + + " {\"add\": {\"index\": \"test_index_reindex\", \"alias\": \"test-system-alias\"}}\n" + + " ]\n" + + "}"); + assertThat(client().performRequest(putAliasRequest).getStatusLine().getStatusCode(), is(200)); + } } else { assertBusy(() -> { Request clusterStateRequest = new Request("GET", "/_cluster/state/metadata"); - Map response = entityAsMap(client().performRequest(clusterStateRequest)); - Map metadata = (Map) response.get("metadata"); - assertNotNull(metadata); - Map indices = (Map) metadata.get("indices"); - assertNotNull(indices); - - Map tasksIndex = (Map) indices.get(".tasks"); - assertNotNull(tasksIndex); + Map indices = new XContentTestUtils.JsonMapView(entityAsMap(client().performRequest(clusterStateRequest))) + .get("metadata.indices"); + + // Make sure our non-system index is still non-system + assertThat(new XContentTestUtils.JsonMapView(indices).get("test_index_old.system"), is(false)); + + // Can't get the .tasks index via JsonMapView because it splits on `.` + assertThat(indices, hasKey(".tasks")); + XContentTestUtils.JsonMapView tasksIndex = new XContentTestUtils.JsonMapView((Map) indices.get(".tasks")); assertThat(tasksIndex.get("system"), is(true)); - Map testIndex = (Map) indices.get("test_index_old"); - assertNotNull(testIndex); - assertThat(testIndex.get("system"), is(false)); + // If .tasks was created in a 7.x version, it should have an alias on it that we need to make sure got upgraded properly. + final String tasksCreatedVersionString = tasksIndex.get("settings.index.version.created"); + assertThat(tasksCreatedVersionString, notNullValue()); + final Version tasksCreatedVersion = Version.fromId(Integer.parseInt(tasksCreatedVersionString)); + if (tasksCreatedVersion.before(SYSTEM_INDEX_ENFORCEMENT_VERSION)) { + // Verify that the alias survived the upgrade + Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); + getAliasRequest.addParameter("allow_system_index_access", "true"); + Map aliasResponse = entityAsMap(client().performRequest(getAliasRequest)); + assertThat(aliasResponse, hasKey(".tasks")); + assertThat(aliasResponse, hasKey("test_index_reindex")); + } }); - - // Verify that the alias survived the upgrade - Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); - getAliasRequest.addParameter("allow_system_index_access", "true"); - Map response = entityAsMap(client().performRequest(getAliasRequest)); - assertThat(response, hasKey(".tasks")); - assertThat(response, hasKey("test_index_reindex")); } } diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java index 26a0813a134eb..6113cd3cc6f07 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java @@ -19,12 +19,18 @@ package org.elasticsearch.upgrades; +import org.elasticsearch.Version; import org.elasticsearch.client.Request; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.test.XContentTestUtils.JsonMapView; import java.util.Map; +import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ENFORCEMENT_VERSION; +import static org.elasticsearch.rest.BaseRestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; public class SystemIndicesUpgradeIT extends AbstractRollingTestCase { @@ -63,45 +69,62 @@ public void testSystemIndicesUpgrades() throws Exception { client().performRequest(getTask); // make sure .tasks index exists + Request getTasksIndex = new Request("GET", "/.tasks"); + getTasksIndex.addParameter("allow_no_indices", "false"); + + // We run "upgrade" tests from the current version as well, so if we're on a recent enough version we need to specify that + // we need system index access here. + if (minimumNodeVersion().onOrAfter(ALLOW_SYSTEM_INDEX_ADDED_VERSION)) { + getTasksIndex.addParameter("allow_system_index_access", "true"); + } assertBusy(() -> { - Request getTasksIndex = new Request("GET", "/.tasks"); - getTasksIndex.addParameter("allow_no_indices", "false"); - assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); + try { + assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); + } catch (ResponseException e) { + throw new AssertionError(".tasks index does not exist yet"); + } }); - // Create an alias to make sure it gets upgraded properly - Request putAliasRequest = new Request("POST", "/_aliases"); - putAliasRequest.setJsonEntity("{\n" + - " \"actions\": [\n" + - " {\"add\": {\"index\": \".tasks\", \"alias\": \"test-system-alias\"}},\n" + - " {\"add\": {\"index\": \"test_index_reindex\", \"alias\": \"test-system-alias\"}}\n" + - " ]\n" + - "}"); - assertThat(client().performRequest(putAliasRequest).getStatusLine().getStatusCode(), is(200)); + // If we are on 7.x create an alias that includes both a system index and a non-system index so we can be sure it gets + // upgraded properly. If we're already on 8.x, skip this part of the test. + if (minimumNodeVersion().before(SYSTEM_INDEX_ENFORCEMENT_VERSION)) { + // Create an alias to make sure it gets upgraded properly + Request putAliasRequest = new Request("POST", "/_aliases"); + putAliasRequest.setJsonEntity("{\n" + + " \"actions\": [\n" + + " {\"add\": {\"index\": \".tasks\", \"alias\": \"test-system-alias\"}},\n" + + " {\"add\": {\"index\": \"test_index_reindex\", \"alias\": \"test-system-alias\"}}\n" + + " ]\n" + + "}"); + assertThat(client().performRequest(putAliasRequest).getStatusLine().getStatusCode(), is(200)); + } } else if (CLUSTER_TYPE == ClusterType.UPGRADED) { assertBusy(() -> { Request clusterStateRequest = new Request("GET", "/_cluster/state/metadata"); - Map response = entityAsMap(client().performRequest(clusterStateRequest)); - Map metadata = (Map) response.get("metadata"); - assertNotNull(metadata); - Map indices = (Map) metadata.get("indices"); - assertNotNull(indices); + Map indices = new JsonMapView(entityAsMap(client().performRequest(clusterStateRequest))) + .get("metadata.indices"); - Map tasksIndex = (Map) indices.get(".tasks"); - assertNotNull(tasksIndex); + // Make sure our non-system index is still non-system + assertThat(new JsonMapView(indices).get("test_index_old.system"), is(false)); + + // Can't get the .tasks index via JsonMapView because it splits on `.` + assertThat(indices, hasKey(".tasks")); + JsonMapView tasksIndex = new JsonMapView((Map) indices.get(".tasks")); assertThat(tasksIndex.get("system"), is(true)); - Map testIndex = (Map) indices.get("test_index_old"); - assertNotNull(testIndex); - assertThat(testIndex.get("system"), is(false)); + // If .tasks was created in a 7.x version, it should have an alias on it that we need to make sure got upgraded properly. + final String tasksCreatedVersionString = tasksIndex.get("settings.index.version.created"); + assertThat(tasksCreatedVersionString, notNullValue()); + final Version tasksCreatedVersion = Version.fromId(Integer.parseInt(tasksCreatedVersionString)); + if (tasksCreatedVersion.before(SYSTEM_INDEX_ENFORCEMENT_VERSION)) { + // Verify that the alias survived the upgrade + Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); + getAliasRequest.addParameter("allow_system_index_access", "true"); + Map aliasResponse = entityAsMap(client().performRequest(getAliasRequest)); + assertThat(aliasResponse, hasKey(".tasks")); + assertThat(aliasResponse, hasKey("test_index_reindex")); + } }); - - // Verify that the alias survived the upgrade - Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); - getAliasRequest.addParameter("allow_system_index_access", "true"); - Map response = entityAsMap(client().performRequest(getAliasRequest)); - assertThat(response, hasKey(".tasks")); - assertThat(response, hasKey("test_index_reindex")); } } } From c742d4a171bd37889f35edf692b7d6ccc003e40e Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 18 Aug 2020 16:48:40 -0600 Subject: [PATCH 056/161] Rename constants to be clearer --- .../cluster/metadata/IndexNameExpressionResolver.java | 4 ++-- .../java/org/elasticsearch/rest/BaseRestHandler.java | 10 +++++----- .../metadata/IndexNameExpressionResolverTests.java | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 4465f4da51e22..f214eef4dbafa 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -64,7 +64,7 @@ public class IndexNameExpressionResolver { public static final String EXCLUDED_DATA_STREAMS_KEY = "es.excluded_ds"; - public static final String SYSTEM_INDEX_ACCESS_CONTROL_KEY = "_system_index_access_allowed"; + public static final String SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY = "_system_index_access_allowed"; public static final Version SYSTEM_INDEX_ENFORCEMENT_VERSION = Version.V_8_0_0; private final DateMathExpressionResolver dateMathExpressionResolver = new DateMathExpressionResolver(); @@ -669,7 +669,7 @@ boolean isPatternMatchingAllIndices(Metadata metadata, String[] indicesOrAliases } private boolean isSystemIndexAccessAllowed() { - return Booleans.parseBoolean(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY), true); + return Booleans.parseBoolean(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY), true); } public static class Context { diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 31cbd7de3421a..6c3054f949e6a 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -44,7 +44,7 @@ import java.util.concurrent.atomic.LongAdder; import java.util.stream.Collectors; -import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_KEY; +import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY; /** * Base handler for REST requests. @@ -59,7 +59,7 @@ public abstract class BaseRestHandler implements RestHandler { public static final Setting MULTI_ALLOW_EXPLICIT_INDEX = Setting.boolSetting("rest.action.multi.allow_explicit_index", true, Property.NodeScope); public static final Version ALLOW_SYSTEM_INDEX_ADDED_VERSION = Version.V_8_0_0; - public static final String ALLOW_SYSTEM_INDEX_ACCESS_KEY = "allow_system_index_access"; + public static final String ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER = "allow_system_index_access"; private final LongAdder usageCount = new LongAdder(); @@ -84,11 +84,11 @@ public final long getUsageCount() { @Override public final void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { if (allowSystemIndexAccessByDefault() == false) { - final String allowSystemIndexParameter = request.param(ALLOW_SYSTEM_INDEX_ACCESS_KEY); + final String allowSystemIndexParameter = request.param(ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER); final ThreadContext threadContext = client.threadPool().getThreadContext(); - if (threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY) == null + if (threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY) == null && Booleans.parseBoolean(allowSystemIndexParameter, false) == false) { - threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, Boolean.FALSE.toString()); + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); } } // prepare the request for execution; has the side effect of touching the request parameters 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 d7690cd08157a..a6730eae9166d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -55,7 +55,7 @@ import static org.elasticsearch.cluster.DataStreamTestHelper.createBackingIndex; import static org.elasticsearch.cluster.DataStreamTestHelper.createTimestampField; import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_HIDDEN_SETTING; -import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_KEY; +import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY; import static org.elasticsearch.common.util.set.Sets.newHashSet; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; @@ -1884,7 +1884,7 @@ public void testSystemIndexResolutionWhenAllowed() { public void testSystemIndexResolutionBlocked() { // Set up the thread context to disallow system index access - threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_KEY, Boolean.FALSE.toString()); + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); Settings settings = Settings.builder().build(); Metadata.Builder mdBuilder = Metadata.builder() From d16ba0e4d94ee5b85719222c66ec0001233b04df Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 18 Aug 2020 16:59:33 -0600 Subject: [PATCH 057/161] Move constants & default access config to `RestHandler` (from `BaseRestHandler`) --- .../java/org/elasticsearch/kibana/KibanaPlugin.java | 2 +- .../upgrades/FullClusterRestartIT.java | 2 +- .../upgrades/SystemIndicesUpgradeIT.java | 2 +- .../org/elasticsearch/http/SystemIndexRestIT.java | 2 +- .../org/elasticsearch/rest/BaseRestHandler.java | 11 ----------- .../java/org/elasticsearch/rest/RestHandler.java | 13 +++++++++++++ .../admin/cluster/RestClusterHealthAction.java | 2 +- .../action/admin/indices/RestRefreshAction.java | 2 +- .../org/elasticsearch/test/rest/ESRestTestCase.java | 2 +- 9 files changed, 20 insertions(+), 18 deletions(-) diff --git a/modules/kibana/src/main/java/org/elasticsearch/kibana/KibanaPlugin.java b/modules/kibana/src/main/java/org/elasticsearch/kibana/KibanaPlugin.java index 016b3bdfa7e48..186da3f827fcb 100644 --- a/modules/kibana/src/main/java/org/elasticsearch/kibana/KibanaPlugin.java +++ b/modules/kibana/src/main/java/org/elasticsearch/kibana/KibanaPlugin.java @@ -134,7 +134,7 @@ public String getName() { } @Override - protected boolean allowSystemIndexAccessByDefault() { + public boolean allowSystemIndexAccessByDefault() { return true; } diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index 6f114abbe31ac..fa576f127a6d5 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -61,7 +61,7 @@ import static org.elasticsearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.rest.BaseRestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; +import static org.elasticsearch.rest.RestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java index 6113cd3cc6f07..95795c5ef4d0c 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java @@ -27,7 +27,7 @@ import java.util.Map; import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ENFORCEMENT_VERSION; -import static org.elasticsearch.rest.BaseRestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; +import static org.elasticsearch.rest.RestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java index ef8b84d18e1c9..d0064dfbfcf44 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -136,7 +136,7 @@ public Collection getSystemIndexDescriptors(Settings sett public static class AddDocRestHandler extends BaseRestHandler { @Override - protected boolean allowSystemIndexAccessByDefault() { + public boolean allowSystemIndexAccessByDefault() { return true; } diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 6c3054f949e6a..fdac90fcb44dd 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -21,7 +21,6 @@ import org.apache.lucene.search.spell.LevenshteinDistance; import org.apache.lucene.util.CollectionUtil; -import org.elasticsearch.Version; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.CheckedConsumer; @@ -58,8 +57,6 @@ public abstract class BaseRestHandler implements RestHandler { public static final Setting MULTI_ALLOW_EXPLICIT_INDEX = Setting.boolSetting("rest.action.multi.allow_explicit_index", true, Property.NodeScope); - public static final Version ALLOW_SYSTEM_INDEX_ADDED_VERSION = Version.V_8_0_0; - public static final String ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER = "allow_system_index_access"; private final LongAdder usageCount = new LongAdder(); @@ -198,14 +195,6 @@ protected Set responseParams() { return Collections.emptySet(); } - /** - * Controls whether requests handled by this class are allowed to to access system indices by default. - * @return {@code true} if requests handled by this class should be allowed to access system indices. - */ - protected boolean allowSystemIndexAccessByDefault() { - return false; - } - public static class Wrapper extends BaseRestHandler { protected final BaseRestHandler delegate; diff --git a/server/src/main/java/org/elasticsearch/rest/RestHandler.java b/server/src/main/java/org/elasticsearch/rest/RestHandler.java index 0c06a84df62bc..1675df9bebce1 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/RestHandler.java @@ -19,6 +19,7 @@ package org.elasticsearch.rest; +import org.elasticsearch.Version; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.rest.RestRequest.Method; @@ -31,6 +32,9 @@ */ public interface RestHandler { + Version ALLOW_SYSTEM_INDEX_ADDED_VERSION = Version.V_8_0_0; + String ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER = "allow_system_index_access"; + /** * Handles a rest request. * @param request The request to handle @@ -89,6 +93,15 @@ default List replacedRoutes() { return Collections.emptyList(); } + + /** + * Controls whether requests handled by this class are allowed to to access system indices by default. + * @return {@code true} if requests handled by this class should be allowed to access system indices. + */ + default boolean allowSystemIndexAccessByDefault() { + return false; + } + class Route { private final String path; diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java index d72aec47eadca..242728e9ed085 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterHealthAction.java @@ -53,7 +53,7 @@ public String getName() { } @Override - protected boolean allowSystemIndexAccessByDefault() { + public boolean allowSystemIndexAccessByDefault() { return true; } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java index ac4060d32c030..3a46318c5e4c8 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java @@ -52,7 +52,7 @@ public String getName() { } @Override - protected boolean allowSystemIndexAccessByDefault() { + public boolean allowSystemIndexAccessByDefault() { return true; } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index e6372042b373a..2d3af91f6f0fb 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -97,7 +97,7 @@ import static java.util.Collections.sort; import static java.util.Collections.unmodifiableList; -import static org.elasticsearch.rest.BaseRestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; +import static org.elasticsearch.rest.RestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; import static org.hamcrest.Matchers.anEmptyMap; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; From 159b40aa6d8000fd5318fd9e99fc346519af4e0c Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 18 Aug 2020 17:03:25 -0600 Subject: [PATCH 058/161] Move header attachment to `RestController#dispatchRequest` (from `BaseRestHandler`) --- .../java/org/elasticsearch/rest/BaseRestHandler.java | 12 ------------ .../java/org/elasticsearch/rest/RestController.java | 11 +++++++++++ 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index fdac90fcb44dd..101c3182fbb62 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -22,12 +22,10 @@ import org.apache.lucene.search.spell.LevenshteinDistance; import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.client.node.NodeClient; -import org.elasticsearch.common.Booleans; import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.rest.action.admin.cluster.RestNodesUsageAction; @@ -43,8 +41,6 @@ import java.util.concurrent.atomic.LongAdder; import java.util.stream.Collectors; -import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY; - /** * Base handler for REST requests. *

@@ -80,14 +76,6 @@ public final long getUsageCount() { @Override public final void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { - if (allowSystemIndexAccessByDefault() == false) { - final String allowSystemIndexParameter = request.param(ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER); - final ThreadContext threadContext = client.threadPool().getThreadContext(); - if (threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY) == null - && Booleans.parseBoolean(allowSystemIndexParameter, false) == false) { - threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); - } - } // prepare the request for execution; has the side effect of touching the request parameters final RestChannelConsumer action = prepareRequest(request, client); diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index a88007398a449..b2ec5a4c0c163 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -24,6 +24,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; @@ -52,7 +53,9 @@ import java.util.function.UnaryOperator; import java.util.stream.Collectors; +import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY; import static org.elasticsearch.rest.BytesRestResponse.TEXT_CONTENT_TYPE; +import static org.elasticsearch.rest.RestHandler.ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER; import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; import static org.elasticsearch.rest.RestStatus.INTERNAL_SERVER_ERROR; import static org.elasticsearch.rest.RestStatus.METHOD_NOT_ALLOWED; @@ -231,6 +234,14 @@ private void dispatchRequest(RestRequest request, RestChannel channel, RestHandl if (handler.allowsUnsafeBuffers() == false) { request.ensureSafeBuffers(); } + if (handler.allowSystemIndexAccessByDefault() == false) { + final String allowSystemIndexParameter = request.param(ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER); + final ThreadContext threadContext = client.threadPool().getThreadContext(); + if (Booleans.parseBoolean(allowSystemIndexParameter, false) == false) { + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); + } + } + handler.handleRequest(request, responseChannel, client); } catch (Exception e) { responseChannel.sendResponse(new BytesRestResponse(responseChannel, e)); From 9b07888124733497f17d4e6bf9b2a5ed2f0be07e Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 19 Aug 2020 15:47:41 -0600 Subject: [PATCH 059/161] Use the correct ThreadContext in RestController --- .../main/java/org/elasticsearch/rest/RestController.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index b2ec5a4c0c163..1f315da72237a 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -207,7 +207,8 @@ public void dispatchBadRequest(final RestChannel channel, final ThreadContext th } } - private void dispatchRequest(RestRequest request, RestChannel channel, RestHandler handler) throws Exception { + private void dispatchRequest(RestRequest request, RestChannel channel, RestHandler handler, + ThreadContext threadContext) throws Exception { final int contentLength = request.contentLength(); if (contentLength > 0) { final XContentType xContentType = request.getXContentType(); @@ -236,7 +237,6 @@ private void dispatchRequest(RestRequest request, RestChannel channel, RestHandl } if (handler.allowSystemIndexAccessByDefault() == false) { final String allowSystemIndexParameter = request.param(ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER); - final ThreadContext threadContext = client.threadPool().getThreadContext(); if (Booleans.parseBoolean(allowSystemIndexParameter, false) == false) { threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); } @@ -324,7 +324,7 @@ private void tryAllHandlers(final RestRequest request, final RestChannel channel return; } } else { - dispatchRequest(request, channel, handler); + dispatchRequest(request, channel, handler, threadContext); return; } } From 46854ab4419c4d626176902780a10c6922ce3727 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 19 Aug 2020 16:05:12 -0600 Subject: [PATCH 060/161] Javadoc per review --- .../elasticsearch/test/client/NoOpClient.java | 9 +++++--- .../test/client/NoOpNodeClient.java | 9 +++++++- .../test/rest/RestActionTestCase.java | 22 +++++++++++++++++++ 3 files changed, 36 insertions(+), 4 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpClient.java b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpClient.java index d034fa0c5e3e3..af7370cdeb537 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpClient.java @@ -20,10 +20,10 @@ package org.elasticsearch.test.client; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionType; import org.elasticsearch.client.support.AbstractClient; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.TestThreadPool; @@ -32,7 +32,10 @@ import java.util.concurrent.TimeUnit; /** - * Client that always responds with {@code null} to every request. Override this for testing. + * Client that always responds with {@code null} to every request. Override {@link #doExecute(ActionType, ActionRequest, ActionListener)} + * for testing. + * + * See also {@link NoOpNodeClient} if you need to mock a {@link org.elasticsearch.client.node.NodeClient}. */ public class NoOpClient extends AbstractClient { /** @@ -43,7 +46,7 @@ public NoOpClient(ThreadPool threadPool) { } /** - * Create a new {@link TestThreadPool} for this client. + * Create a new {@link TestThreadPool} for this client. This {@linkplain TestThreadPool} is terminated on {@link #close()}. */ public NoOpClient(String testName) { super(Settings.EMPTY, new TestThreadPool(testName)); diff --git a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java index b75766fb3142f..5ec03f67e5dc6 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java @@ -39,6 +39,13 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; +/** + * Client that always response with {@code null} to every request. Override {@link #doExecute(ActionType, ActionRequest, ActionListener)}, + * {@link #executeLocally(ActionType, ActionRequest, ActionListener)}, or {@link #executeLocally(ActionType, ActionRequest, TaskListener)} + * for testing. + * + * See also {@link NoOpClient} if you do not specifically need a {@link NodeClient}. + */ public class NoOpNodeClient extends NodeClient { /** @@ -49,7 +56,7 @@ public NoOpNodeClient(ThreadPool threadPool) { } /** - * Create a new {@link TestThreadPool} for this client. + * Create a new {@link TestThreadPool} for this client. This {@linkplain TestThreadPool} is terminated on {@link #close()}. */ public NoOpNodeClient(String testName) { super(Settings.EMPTY, new TestThreadPool(testName)); diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/RestActionTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/RestActionTestCase.java index 8adaccfe07a56..0577ad0c23441 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/RestActionTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/RestActionTestCase.java @@ -79,6 +79,13 @@ protected void dispatchRequest(RestRequest request) { controller.dispatchRequest(request, channel, threadContext); } + /** + * A mocked {@link org.elasticsearch.client.node.NodeClient} which can be easily reconfigured to verify arbitrary verification + * functions, and can be reset to allow reconfiguration partway through a test without having to construct a new object. + * + * By default, will throw {@link AssertionError} when any execution method is called, unless configured otherwise using + * {@link #setExecuteVerifier(BiFunction)} or {@link #setExecuteLocallyVerifier(BiFunction)}. + */ public static class VerifyingClient extends NoOpNodeClient { AtomicReference executeVerifier = new AtomicReference<>(); AtomicReference executeLocallyVerifier = new AtomicReference<>(); @@ -88,6 +95,11 @@ public VerifyingClient(String testName) { reset(); } + /** + * Clears any previously set verifier functions set by {@link #setExecuteVerifier(BiFunction)} and/or + * {@link #setExecuteLocallyVerifier(BiFunction)}. These functions are replaced with functions which will throw an + * {@link AssertionError} if called. + */ public void reset() { executeVerifier.set((arg1, arg2) -> { throw new AssertionError(); @@ -97,6 +109,11 @@ public void reset() { }); } + /** + * Sets the function that will be called when {@link #doExecute(ActionType, ActionRequest, ActionListener)} is called. The given + * function should return either a subclass of {@link ActionResponse} or {@code null}. + * @param verifier A function which is called in place of {@link #doExecute(ActionType, ActionRequest, ActionListener)} + */ public void setExecuteVerifier(BiFunction, Request, Void> verifier) { executeVerifier.set(verifier); @@ -108,6 +125,11 @@ void doExecute(ActionType action, Request request, ActionListener void setExecuteLocallyVerifier(BiFunction, Request, Void> verifier) { executeLocallyVerifier.set(verifier); From 30225b302cc3a733a44f68f26eb694d406fa0164 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 20 Aug 2020 14:56:17 -0600 Subject: [PATCH 061/161] Ensure allowSystemIndexAccessByDefault is passed through SecurityRestFilter --- .../xpack/security/rest/SecurityRestFilter.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/SecurityRestFilter.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/SecurityRestFilter.java index d2327dcb3e833..0ba39ae2d26f6 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/SecurityRestFilter.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/SecurityRestFilter.java @@ -50,6 +50,11 @@ public SecurityRestFilter(XPackLicenseState licenseState, ThreadContext threadCo this.extractClientCertificate = extractClientCertificate; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return restHandler.allowSystemIndexAccessByDefault(); + } + @Override public void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { if (licenseState.isSecurityEnabled() && request.method() != Method.OPTIONS) { From 0bf65e4106a8446267c01ff1bf3ec56954598f0d Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 20 Aug 2020 17:15:01 -0600 Subject: [PATCH 062/161] Switch to emitting a deprecation warning instead of changing behavior --- .../elasticsearch/http/SystemIndexRestIT.java | 30 +++++++++++-------- .../metadata/IndexNameExpressionResolver.java | 25 +++++++++++----- .../IndexNameExpressionResolverTests.java | 16 +++++++--- 3 files changed, 47 insertions(+), 24 deletions(-) diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java index d0064dfbfcf44..3530f0730d84e 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -22,8 +22,8 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.Response; -import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNodes; @@ -88,33 +88,39 @@ public void testSystemIndexAccessBlockedByDefault() throws Exception { }); // now try without `allow_system_index_access` - assertAccessBlocked(SystemIndexTestPlugin.SYSTEM_INDEX_NAME); + assertDeprecationWarningOnAccess(SystemIndexTestPlugin.SYSTEM_INDEX_NAME, SystemIndexTestPlugin.SYSTEM_INDEX_NAME); // And with a partial wildcard - assertAccessBlocked(".test-*"); + assertDeprecationWarningOnAccess(".test-*", SystemIndexTestPlugin.SYSTEM_INDEX_NAME); // And with a total wildcard - assertAccessBlocked(randomFrom("*", "_all")); + assertDeprecationWarningOnAccess(randomFrom("*", "_all"), SystemIndexTestPlugin.SYSTEM_INDEX_NAME); // Try to index a doc directly { + String expectedWarning = "this request accesses system indices: [" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + "], but in a " + + "future major version, direct access to system indices will be prevented by default"; + RequestOptions expectWarningOptions = RequestOptions.DEFAULT.toBuilder().setWarningsHandler(expectedWarning::equals).build(); Request putDocDirectlyRequest = new Request("PUT", "/" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + "/_doc/43"); putDocDirectlyRequest.setJsonEntity("{\"some_field\": \"some_other_value\"}"); - ResponseException exception = expectThrows(ResponseException.class, - () -> getRestClient().performRequest(putDocDirectlyRequest)); - logger.info("{}", entityAsMap(exception.getResponse())); - assertThat(exception.getResponse().getStatusLine().getStatusCode(), equalTo(404)); + putDocDirectlyRequest.setOptions(expectWarningOptions); + Response response = getRestClient().performRequest(putDocDirectlyRequest); + assertThat(response.getStatusLine().getStatusCode(), equalTo(201)); } } - private void assertAccessBlocked(String s) { - Request searchRequest = new Request("GET", "/" + s + randomFrom("/_count", "/_search")); + private void assertDeprecationWarningOnAccess(String queryPattern, String warningIndexName) throws IOException { + String expectedWarning = "this request accesses system indices: [" + warningIndexName + "], but in a " + + "future major version, direct access to system indices will be prevented by default"; + RequestOptions expectWarningOptions = RequestOptions.DEFAULT.toBuilder().setWarningsHandler(expectedWarning::equals).build(); + Request searchRequest = new Request("GET", "/" + queryPattern + randomFrom("/_count", "/_search")); searchRequest.setJsonEntity("{\"query\": {\"match\": {\"some_field\": \"some_value\"}}}"); // Disallow no indices to cause an exception if this resolves to zero indices (as we expect) searchRequest.addParameter("allow_no_indices", "false"); + searchRequest.setOptions(expectWarningOptions); - ResponseException exception = expectThrows(ResponseException.class, () -> getRestClient().performRequest(searchRequest)); - assertThat(exception.getResponse().getStatusLine().getStatusCode(), equalTo(404)); + Response response = getRestClient().performRequest(searchRequest); + assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); } public static class SystemIndexTestPlugin extends Plugin implements SystemIndexPlugin { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index f214eef4dbafa..18234d64af060 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.common.time.DateMathParser; @@ -62,6 +63,7 @@ import java.util.stream.StreamSupport; public class IndexNameExpressionResolver { + public static DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(IndexNameExpressionResolver.class); public static final String EXCLUDED_DATA_STREAMS_KEY = "es.excluded_ds"; public static final String SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY = "_system_index_access_allowed"; @@ -296,13 +298,22 @@ Index[] concreteIndices(Context context, String... indexExpressions) { } throw infe; } + if (context.isSystemIndexAccessAllowed() == false) { + final List resolvedSystemIndices = concreteIndices.stream() + .map(metadata::index) + .filter(IndexMetadata::isSystem) + .map(i -> i.getIndex().getName()) + .sorted() // reliable order for testing + .collect(Collectors.toList()); + if (resolvedSystemIndices.isEmpty() == false) { + deprecationLogger.deprecate("open_system_index_access", "this request accesses system indices: {}, but in a future" + + " major version, direct access to system indices will be prevented by default", resolvedSystemIndices); + } + } return concreteIndices.toArray(new Index[concreteIndices.size()]); } private static boolean shouldTrackConcreteIndex(Context context, IndicesOptions options, IndexMetadata index) { - if (context.isSystemIndexAccessAllowed() == false && index.isSystem()) { - return false; - } if (index.getState() == IndexMetadata.State.CLOSE) { if (options.forbidClosedIndices() && options.ignoreUnavailable() == false) { @@ -752,6 +763,9 @@ public boolean isPreserveDataStreams() { return preserveDataStreams; } + /** + * Used to determine if it is allowed to access system indices in this context (e.g. for this request). + */ public boolean isSystemIndexAccessAllowed() { return isSystemIndexAccessAllowed; } @@ -851,8 +865,6 @@ private Set innerResolve(Context context, List expressions, Indi } else if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM && context.includeDataStreams() == false) { throw indexNotFoundException(expression); - } else if (indexAbstraction.isSystem() && context.isSystemIndexAccessAllowed() == false) { - throw indexNotFoundException(expression); } } if (add) { @@ -986,9 +998,6 @@ private static Set expand(Context context, IndexMetadata.State excludeSt for (Map.Entry entry : matches.entrySet()) { String aliasOrIndexName = entry.getKey(); IndexAbstraction indexAbstraction = entry.getValue(); - if (context.isSystemIndexAccessAllowed() == false && indexAbstraction.isSystem()) { - continue; - } if (indexAbstraction.isHidden() == false || includeHidden || implicitHiddenMatch(aliasOrIndexName, expression)) { if (context.isPreserveAliases() && indexAbstraction.getType() == IndexAbstraction.Type.ALIAS) { 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 a6730eae9166d..eecd346583832 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -1902,7 +1902,9 @@ public void testSystemIndexResolutionBlocked() { .stream(indexNameExpressionResolver.concreteIndices(state, request)) .map(i -> i.getName()) .collect(Collectors.toList()); - assertThat(indexNames, empty()); + assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); + assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff], but in a future major version, direct access " + + "to system indices will be prevented by default"); } // Wilcard that might match a single index @@ -1913,7 +1915,9 @@ public void testSystemIndexResolutionBlocked() { .stream(indexNameExpressionResolver.concreteIndices(state, request)) .map(i -> i.getName()) .collect(Collectors.toList()); - assertThat(indexNames, empty()); + assertThat(indexNames, containsInAnyOrder(".watches")); + assertWarnings("this request accesses system indices: [.watches], but in a future major version, direct access " + + "to system indices will be prevented by default"); } // A specific index name @@ -1924,7 +1928,9 @@ public void testSystemIndexResolutionBlocked() { .stream(indexNameExpressionResolver.concreteIndices(state, request)) .map(i -> i.getName()) .collect(Collectors.toList()); - assertThat(indexNames, empty()); + assertThat(indexNames, containsInAnyOrder(".ml-meta")); + assertWarnings("this request accesses system indices: [.ml-meta], but in a future major version, direct access " + + "to system indices will be prevented by default"); } // Full wildcard @@ -1935,7 +1941,9 @@ public void testSystemIndexResolutionBlocked() { .stream(indexNameExpressionResolver.concreteIndices(state, request)) .map(i -> i.getName()) .collect(Collectors.toList()); - assertThat(indexNames, containsInAnyOrder("some-other-index")); + assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); + assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff, .watches], but in a future major version, direct " + + "access to system indices will be prevented by default"); } } From 515b4eb78a18abd026eb890e9594d478e368ef43 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 25 Aug 2020 16:01:28 -0600 Subject: [PATCH 063/161] Correct NoOpNodeClient for changes in NodeClient --- .../java/org/elasticsearch/test/client/NoOpNodeClient.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java index 5ec03f67e5dc6..4edc7e6114971 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpNodeClient.java @@ -27,6 +27,7 @@ import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.client.Client; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskListener; @@ -70,7 +71,7 @@ void doExecute(ActionType action, Request request, ActionListener actions, TaskManager taskManager, Supplier localNodeId, - RemoteClusterService remoteClusterService) { + RemoteClusterService remoteClusterService, NamedWriteableRegistry namedWriteableRegistry) { throw new UnsupportedOperationException("cannot initialize " + this.getClass().getSimpleName()); } From 39538e3151f662a76a6880c53bbac16617374f3f Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 26 Aug 2020 18:21:54 -0600 Subject: [PATCH 064/161] Implement "feature flag" for choosing system index access behavior --- qa/smoke-test-http/build.gradle | 5 +++ .../elasticsearch/http/SystemIndexRestIT.java | 4 +- server/build.gradle | 7 ++++ .../metadata/IndexNameExpressionResolver.java | 39 +++++++++++++++++-- .../elasticsearch/indices/SystemIndices.java | 34 ++++++++++++++++ 5 files changed, 84 insertions(+), 5 deletions(-) diff --git a/qa/smoke-test-http/build.gradle b/qa/smoke-test-http/build.gradle index 6f3fb39b70c1b..d14c92dfc595c 100644 --- a/qa/smoke-test-http/build.gradle +++ b/qa/smoke-test-http/build.gradle @@ -34,4 +34,9 @@ integTest { * other if we allow them to set the number of available processors as it's set-once in Netty. */ systemProperty 'es.set.netty.runtime.available.processors', 'false' + + /* We need to set this here to prevent release builds from failing while the default is different between snapshot and release builds. + * Specifically, this is needed for SystemIndexRestIT, which assumes the value is `deprecated`. + */ + systemProperty 'es.system_index_access_behavior', 'deprecated' } diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java index 3530f0730d84e..d479e6c2c99de 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -112,7 +112,9 @@ public void testSystemIndexAccessBlockedByDefault() throws Exception { private void assertDeprecationWarningOnAccess(String queryPattern, String warningIndexName) throws IOException { String expectedWarning = "this request accesses system indices: [" + warningIndexName + "], but in a " + "future major version, direct access to system indices will be prevented by default"; - RequestOptions expectWarningOptions = RequestOptions.DEFAULT.toBuilder().setWarningsHandler(expectedWarning::equals).build(); + RequestOptions expectWarningOptions = RequestOptions.DEFAULT.toBuilder() + .setWarningsHandler(w -> w.contains(expectedWarning) == false || w.size() != 1) + .build(); Request searchRequest = new Request("GET", "/" + queryPattern + randomFrom("/_count", "/_search")); searchRequest.setJsonEntity("{\"query\": {\"match\": {\"some_field\": \"some_value\"}}}"); // Disallow no indices to cause an exception if this resolves to zero indices (as we expect) diff --git a/server/build.gradle b/server/build.gradle index fb28a10e0dc84..1ae0570679d54 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -298,3 +298,10 @@ tasks.named("licenseHeaders").configure { excludes << 'org/apache/lucene/search/RegExp87*' excludes << 'org/apache/lucene/search/RegexpQuery87*' } + +tasks.named("test").configure { + /* We need to set this to prevent release builds from failing while the default is different between snapshot and release builds. + * In particular, this is needed for IndexNameExpressionResolverTests, which assumes the value is `deprecated`. + */ + systemProperty 'es.system_index_access_behavior', 'deprecated' +} diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 18234d64af060..f238a1819abc7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -41,6 +41,7 @@ import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.indices.IndexClosedException; import org.elasticsearch.indices.InvalidIndexNameException; +import org.elasticsearch.indices.SystemIndices; import java.time.Instant; import java.time.ZoneId; @@ -62,6 +63,8 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; +import static org.elasticsearch.indices.SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR; + public class IndexNameExpressionResolver { public static DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(IndexNameExpressionResolver.class); @@ -298,6 +301,11 @@ Index[] concreteIndices(Context context, String... indexExpressions) { } throw infe; } + checkSystemIndexAccess(context, metadata, concreteIndices); + return concreteIndices.toArray(new Index[concreteIndices.size()]); + } + + private static void checkSystemIndexAccess(Context context, Metadata metadata, Set concreteIndices) { if (context.isSystemIndexAccessAllowed() == false) { final List resolvedSystemIndices = concreteIndices.stream() .map(metadata::index) @@ -306,15 +314,31 @@ Index[] concreteIndices(Context context, String... indexExpressions) { .sorted() // reliable order for testing .collect(Collectors.toList()); if (resolvedSystemIndices.isEmpty() == false) { - deprecationLogger.deprecate("open_system_index_access", "this request accesses system indices: {}, but in a future" + - " major version, direct access to system indices will be prevented by default", resolvedSystemIndices); + switch (SYSTEM_INDEX_ACCESS_BEHAVIOR) { + case ALLOWED: + // If access is allowed in all cases, then there's nothing to do here. + break; + case DEPRECATED: + deprecationLogger.deprecate("open_system_index_access", + "this request accesses system indices: {}, but in a future major version, direct access to system " + + "indices will be prevented by default", resolvedSystemIndices); + break; + case RESTRICTED: + final String errorMsg = "while resolving indices, resolved system indices [" + resolvedSystemIndices + + "] when access to system indices is restricted"; + assert false : errorMsg; + throw new IllegalArgumentException(errorMsg); + } + ; } } - return concreteIndices.toArray(new Index[concreteIndices.size()]); } private static boolean shouldTrackConcreteIndex(Context context, IndicesOptions options, IndexMetadata index) { - + if (SYSTEM_INDEX_ACCESS_BEHAVIOR == SystemIndices.AccessBehavior.RESTRICTED + && context.isSystemIndexAccessAllowed() == false && index.isSystem()) { + return false; + } if (index.getState() == IndexMetadata.State.CLOSE) { if (options.forbidClosedIndices() && options.ignoreUnavailable() == false) { throw new IndexClosedException(index.getIndex()); @@ -865,6 +889,9 @@ private Set innerResolve(Context context, List expressions, Indi } else if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM && context.includeDataStreams() == false) { throw indexNotFoundException(expression); + } else if (SYSTEM_INDEX_ACCESS_BEHAVIOR == SystemIndices.AccessBehavior.RESTRICTED + && context.isSystemIndexAccessAllowed() == false && indexAbstraction.isSystem()) { + throw indexNotFoundException(expression); } } if (add) { @@ -998,6 +1025,10 @@ private static Set expand(Context context, IndexMetadata.State excludeSt for (Map.Entry entry : matches.entrySet()) { String aliasOrIndexName = entry.getKey(); IndexAbstraction indexAbstraction = entry.getValue(); + if (SYSTEM_INDEX_ACCESS_BEHAVIOR == SystemIndices.AccessBehavior.RESTRICTED + && context.isSystemIndexAccessAllowed() == false && indexAbstraction.isSystem()) { + continue; + } if (indexAbstraction.isHidden() == false || includeHidden || implicitHiddenMatch(aliasOrIndexName, expression)) { if (context.isPreserveAliases() && indexAbstraction.getType() == IndexAbstraction.Type.ALIAS) { diff --git a/server/src/main/java/org/elasticsearch/indices/SystemIndices.java b/server/src/main/java/org/elasticsearch/indices/SystemIndices.java index 71a21b277763c..a17d3357b8627 100644 --- a/server/src/main/java/org/elasticsearch/indices/SystemIndices.java +++ b/server/src/main/java/org/elasticsearch/indices/SystemIndices.java @@ -24,6 +24,7 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton; import org.apache.lucene.util.automaton.MinimizationOperations; import org.apache.lucene.util.automaton.Operations; +import org.elasticsearch.Build; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.regex.Regex; @@ -34,6 +35,7 @@ import java.util.Comparator; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -47,6 +49,22 @@ * to reduce the locations within the code that need to deal with {@link SystemIndexDescriptor}s. */ public class SystemIndices { + public static final AccessBehavior SYSTEM_INDEX_ACCESS_BEHAVIOR; + + static { + final String property = System.getProperty("es.system_index_access_behavior"); + if (property == null) { + SYSTEM_INDEX_ACCESS_BEHAVIOR = Build.CURRENT.isSnapshot() ? AccessBehavior.DEPRECATED : AccessBehavior.ALLOWED; + } else { + try { + SYSTEM_INDEX_ACCESS_BEHAVIOR = AccessBehavior.fromString(property); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException( + "expected es.system_index_access_behavior to be unset or [allowed|deprecated|restricted] but was [" + property + "]", e + ); + } + } + } private static final Map> SERVER_SYSTEM_INDEX_DESCRIPTORS = Map.of( TaskResultsService.class.getName(), List.of(new SystemIndexDescriptor(TASK_INDEX + "*", "Task Result Index")) @@ -172,4 +190,20 @@ private static Map> buildSystemIndexDe }); return Map.copyOf(map); } + + public enum AccessBehavior { + ALLOWED, + DEPRECATED, + RESTRICTED; + + public static AccessBehavior fromString(String value) { + return AccessBehavior.valueOf(value.toUpperCase(Locale.ROOT)); + } + + + @Override + public String toString() { + return name().toLowerCase(Locale.ROOT); + } + } } From e75c50fdce607d93e6af862c5170b493c8080533 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 26 Aug 2020 18:51:54 -0600 Subject: [PATCH 065/161] Docs changes per review --- docs/reference/api-conventions.asciidoc | 3 ++- docs/reference/docs/delete-by-query.asciidoc | 4 ++++ docs/reference/docs/update-by-query.asciidoc | 4 ++++ docs/reference/index-modules/blocks.asciidoc | 4 ++++ docs/reference/indices/alias-exists.asciidoc | 4 ++++ docs/reference/indices/apis/reload-analyzers.asciidoc | 4 ++++ docs/reference/indices/clearcache.asciidoc | 4 ++++ docs/reference/indices/close.asciidoc | 4 ++++ docs/reference/indices/delete-index.asciidoc | 4 ++++ docs/reference/indices/flush.asciidoc | 4 ++++ docs/reference/indices/forcemerge.asciidoc | 4 ++++ docs/reference/indices/get-alias.asciidoc | 4 ++++ docs/reference/indices/get-field-mapping.asciidoc | 4 ++++ docs/reference/indices/get-index.asciidoc | 4 ++++ docs/reference/indices/get-mapping.asciidoc | 4 ++++ docs/reference/indices/get-settings.asciidoc | 4 ++++ docs/reference/indices/indices-exists.asciidoc | 4 ++++ docs/reference/indices/open-close.asciidoc | 4 ++++ docs/reference/indices/put-mapping.asciidoc | 4 ++++ docs/reference/indices/refresh.asciidoc | 4 ++++ docs/reference/indices/resolve.asciidoc | 4 ++++ docs/reference/indices/segments.asciidoc | 4 ++++ docs/reference/indices/shard-stores.asciidoc | 4 ++++ docs/reference/indices/stats.asciidoc | 4 ++++ docs/reference/indices/update-settings.asciidoc | 4 ++++ docs/reference/rest-api/common-parms.asciidoc | 1 - docs/reference/search/count.asciidoc | 4 ++++ docs/reference/search/field-caps.asciidoc | 4 ++++ docs/reference/search/multi-search.asciidoc | 8 ++++++++ docs/reference/search/rank-eval.asciidoc | 4 ++++ docs/reference/search/search-shards.asciidoc | 4 ++++ docs/reference/search/search-template.asciidoc | 4 ++++ docs/reference/search/search.asciidoc | 6 +++++- docs/reference/search/validate.asciidoc | 4 ++++ 34 files changed, 135 insertions(+), 3 deletions(-) diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index feb6c52ed1890..3126b052c169c 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -47,7 +47,8 @@ indices by using the following query string parameter: include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] Using this parameter will prevent deprecation warnings in the current version, -and maintain the current behavior in the next major version. +and maintain the current behavior with deprecation warnings in the next major +version. Some multi-target APIs that can target indices also support the following query string parameter: diff --git a/docs/reference/docs/delete-by-query.asciidoc b/docs/reference/docs/delete-by-query.asciidoc index 55ebed7077ffb..ea77fdc48e89a 100644 --- a/docs/reference/docs/delete-by-query.asciidoc +++ b/docs/reference/docs/delete-by-query.asciidoc @@ -185,6 +185,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=from] diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index f75ebd56ac6f0..43904bf5896ac 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -180,6 +180,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=from] diff --git a/docs/reference/index-modules/blocks.asciidoc b/docs/reference/index-modules/blocks.asciidoc index c35ed389abc16..9ffdf0ceefa29 100644 --- a/docs/reference/index-modules/blocks.asciidoc +++ b/docs/reference/index-modules/blocks.asciidoc @@ -130,6 +130,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/alias-exists.asciidoc b/docs/reference/indices/alias-exists.asciidoc index 0139d78e0081e..1e039a1443d94 100644 --- a/docs/reference/indices/alias-exists.asciidoc +++ b/docs/reference/indices/alias-exists.asciidoc @@ -41,6 +41,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `all`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/apis/reload-analyzers.asciidoc b/docs/reference/indices/apis/reload-analyzers.asciidoc index 029bd617eccff..f8c272f9efc1e 100644 --- a/docs/reference/indices/apis/reload-analyzers.asciidoc +++ b/docs/reference/indices/apis/reload-analyzers.asciidoc @@ -86,6 +86,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/clearcache.asciidoc b/docs/reference/indices/clearcache.asciidoc index 342f5740c1fc3..5a281b560d66f 100644 --- a/docs/reference/indices/clearcache.asciidoc +++ b/docs/reference/indices/clearcache.asciidoc @@ -46,6 +46,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- `fielddata`:: + diff --git a/docs/reference/indices/close.asciidoc b/docs/reference/indices/close.asciidoc index 9a8b6ce95e42f..2d19eb6b3be0e 100644 --- a/docs/reference/indices/close.asciidoc +++ b/docs/reference/indices/close.asciidoc @@ -51,6 +51,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/delete-index.asciidoc b/docs/reference/indices/delete-index.asciidoc index 64d8dac0cf301..b8d241c236fba 100644 --- a/docs/reference/indices/delete-index.asciidoc +++ b/docs/reference/indices/delete-index.asciidoc @@ -56,6 +56,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index e250d8a5cac04..5fabeb7f28313 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -73,6 +73,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- `force`:: + diff --git a/docs/reference/indices/forcemerge.asciidoc b/docs/reference/indices/forcemerge.asciidoc index 5e34ef91ebab1..ac2bba0ebca2e 100644 --- a/docs/reference/indices/forcemerge.asciidoc +++ b/docs/reference/indices/forcemerge.asciidoc @@ -90,6 +90,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- `flush`:: (Optional, boolean) diff --git a/docs/reference/indices/get-alias.asciidoc b/docs/reference/indices/get-alias.asciidoc index 8194744abf7f5..dcad9a7141431 100644 --- a/docs/reference/indices/get-alias.asciidoc +++ b/docs/reference/indices/get-alias.asciidoc @@ -51,6 +51,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `all`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/get-field-mapping.asciidoc b/docs/reference/indices/get-field-mapping.asciidoc index 83d3175935153..a38e1af55fd4b 100644 --- a/docs/reference/indices/get-field-mapping.asciidoc +++ b/docs/reference/indices/get-field-mapping.asciidoc @@ -50,6 +50,10 @@ Defaults to `true`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/get-index.asciidoc b/docs/reference/indices/get-index.asciidoc index dcf7481bd4240..d46802114dc66 100644 --- a/docs/reference/indices/get-index.asciidoc +++ b/docs/reference/indices/get-index.asciidoc @@ -43,6 +43,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] diff --git a/docs/reference/indices/get-mapping.asciidoc b/docs/reference/indices/get-mapping.asciidoc index 7d37029235063..528e132751b79 100644 --- a/docs/reference/indices/get-mapping.asciidoc +++ b/docs/reference/indices/get-mapping.asciidoc @@ -45,6 +45,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/get-settings.asciidoc b/docs/reference/indices/get-settings.asciidoc index fe12cee8e8dc0..ded5cfd93ebe5 100644 --- a/docs/reference/indices/get-settings.asciidoc +++ b/docs/reference/indices/get-settings.asciidoc @@ -50,6 +50,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `all`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] diff --git a/docs/reference/indices/indices-exists.asciidoc b/docs/reference/indices/indices-exists.asciidoc index 264c92cf8336a..c1f3bf04eb3bb 100644 --- a/docs/reference/indices/indices-exists.asciidoc +++ b/docs/reference/indices/indices-exists.asciidoc @@ -42,6 +42,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] diff --git a/docs/reference/indices/open-close.asciidoc b/docs/reference/indices/open-close.asciidoc index dc60cee63ed23..46e8c370c02af 100644 --- a/docs/reference/indices/open-close.asciidoc +++ b/docs/reference/indices/open-close.asciidoc @@ -107,6 +107,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `closed`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/put-mapping.asciidoc b/docs/reference/indices/put-mapping.asciidoc index 9d06ce8c87010..ad560bb1efc2a 100644 --- a/docs/reference/indices/put-mapping.asciidoc +++ b/docs/reference/indices/put-mapping.asciidoc @@ -54,6 +54,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/refresh.asciidoc b/docs/reference/indices/refresh.asciidoc index 109799f48347a..7063c79ddd57e 100644 --- a/docs/reference/indices/refresh.asciidoc +++ b/docs/reference/indices/refresh.asciidoc @@ -85,6 +85,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `true`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/resolve.asciidoc b/docs/reference/indices/resolve.asciidoc index f38fd3fd00100..218856f77f9b7 100644 --- a/docs/reference/indices/resolve.asciidoc +++ b/docs/reference/indices/resolve.asciidoc @@ -77,6 +77,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- [[resolve-index-api-example]] ==== {api-examples-title} diff --git a/docs/reference/indices/segments.asciidoc b/docs/reference/indices/segments.asciidoc index 76cdd9debddae..c5c6bad3ef5c5 100644 --- a/docs/reference/indices/segments.asciidoc +++ b/docs/reference/indices/segments.asciidoc @@ -47,6 +47,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/shard-stores.asciidoc b/docs/reference/indices/shard-stores.asciidoc index f0c6717225d65..e4cf7e7ce05bc 100644 --- a/docs/reference/indices/shard-stores.asciidoc +++ b/docs/reference/indices/shard-stores.asciidoc @@ -65,6 +65,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/stats.asciidoc b/docs/reference/indices/stats.asciidoc index 99eff6e0a6540..bf36c304e3c34 100644 --- a/docs/reference/indices/stats.asciidoc +++ b/docs/reference/indices/stats.asciidoc @@ -72,6 +72,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=fields] diff --git a/docs/reference/indices/update-settings.asciidoc b/docs/reference/indices/update-settings.asciidoc index 60bce615e6abe..e5ccb9bb335c3 100644 --- a/docs/reference/indices/update-settings.asciidoc +++ b/docs/reference/indices/update-settings.asciidoc @@ -51,6 +51,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] diff --git a/docs/reference/rest-api/common-parms.asciidoc b/docs/reference/rest-api/common-parms.asciidoc index 42a36e0eab0b6..40a9eb5f9cfd4 100644 --- a/docs/reference/rest-api/common-parms.asciidoc +++ b/docs/reference/rest-api/common-parms.asciidoc @@ -233,7 +233,6 @@ tag::allow-system-index-access[] (Optional, boolean) If `true`, allows this request to access system indices directly. This option should be used with caution, as system indices are not intended to be accessed directly. -Defaults to `false`. end::allow-system-index-access[] tag::exponential-avg-checkpoint-duration-ms[] diff --git a/docs/reference/search/count.asciidoc b/docs/reference/search/count.asciidoc index 57e9ffbeb7662..16e3cee2cc10b 100644 --- a/docs/reference/search/count.asciidoc +++ b/docs/reference/search/count.asciidoc @@ -67,6 +67,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=ignore_throttled] diff --git a/docs/reference/search/field-caps.asciidoc b/docs/reference/search/field-caps.asciidoc index 6a70e86e4439e..fcc1b4e0da241 100644 --- a/docs/reference/search/field-caps.asciidoc +++ b/docs/reference/search/field-caps.asciidoc @@ -57,6 +57,10 @@ Defaults to `open`. -- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=fields] diff --git a/docs/reference/search/multi-search.asciidoc b/docs/reference/search/multi-search.asciidoc index 9fb7470ce485d..968f22a0075d1 100644 --- a/docs/reference/search/multi-search.asciidoc +++ b/docs/reference/search/multi-search.asciidoc @@ -79,6 +79,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- `ignore_throttled`:: (Optional, boolean) @@ -186,6 +190,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- `ignore_unavailable`::: (Optional, boolean) If `true`, documents from missing or closed indices are not diff --git a/docs/reference/search/rank-eval.asciidoc b/docs/reference/search/rank-eval.asciidoc index c90644c81b478..659ce1560c776 100644 --- a/docs/reference/search/rank-eval.asciidoc +++ b/docs/reference/search/rank-eval.asciidoc @@ -86,6 +86,10 @@ Defaults to `open`. -- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/search/search-shards.asciidoc b/docs/reference/search/search-shards.asciidoc index 6cf6843df2728..f1550f78c2028 100644 --- a/docs/reference/search/search-shards.asciidoc +++ b/docs/reference/search/search-shards.asciidoc @@ -45,6 +45,10 @@ Defaults to `open`. -- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/search/search-template.asciidoc b/docs/reference/search/search-template.asciidoc index bf56509c957b0..a864a5c41dd5f 100644 --- a/docs/reference/search/search-template.asciidoc +++ b/docs/reference/search/search-template.asciidoc @@ -63,6 +63,10 @@ Defaults to `true`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- `explain`:: (Optional, boolean) If `true`, the response includes additional details about diff --git a/docs/reference/search/search.asciidoc b/docs/reference/search/search.asciidoc index 4bf3760eba3a6..34ff00e5b03d2 100644 --- a/docs/reference/search/search.asciidoc +++ b/docs/reference/search/search.asciidoc @@ -84,6 +84,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- `explain`:: (Optional, boolean) If `true`, returns detailed information about score @@ -399,7 +403,7 @@ are supported. `` is the factor by which scores are multiplied. + A boost value greater than `1.0` increases the score. A boost value between -`0` and `1.0` decreases the score. +`0` and `1.0` decreases the score. ==== [[search-api-min-score]] diff --git a/docs/reference/search/validate.asciidoc b/docs/reference/search/validate.asciidoc index 674c001c707b9..b98bb5f6cc7b7 100644 --- a/docs/reference/search/validate.asciidoc +++ b/docs/reference/search/validate.asciidoc @@ -60,6 +60,10 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=df] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] ++ +-- +Defaults to `false`. +-- `explain`:: (Optional, boolean) If `true`, the response returns detailed information if an From 06c7ca402e47532ba8ae5312e65117ce3cc4a1ed Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 26 Aug 2020 18:52:06 -0600 Subject: [PATCH 066/161] Formatting changes per review --- .../cluster/metadata/DateMathExpressionResolverTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java index fbef1a216c8de..35b113b1c751f 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java @@ -44,7 +44,8 @@ public class DateMathExpressionResolverTests extends ESTestCase { private final DateMathExpressionResolver expressionResolver = new DateMathExpressionResolver(); private final Context context = new Context( ClusterState.builder(new ClusterName("_name")).build(), IndicesOptions.strictExpand(), - false); + false + ); public void testNormal() throws Exception { int numIndexExpressions = randomIntBetween(1, 9); From 966cfc38e7452aa46026add4e6911c6c44202a01 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 27 Aug 2020 16:12:49 -0600 Subject: [PATCH 067/161] allow_system_index_access flag in YAML tests as necessary --- .../test/resources/rest-api-spec/test/ml/custom_all_field.yml | 1 + .../src/test/resources/rest-api-spec/test/ml/index_layout.yml | 3 +++ 2 files changed, 4 insertions(+) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml index f6bf53fb289db..fd3c1d601f135 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml @@ -148,6 +148,7 @@ setup: - do: search: + allow_system_index_access: true expand_wildcards: all rest_total_hits_as_int: true body: { query: { bool: { must: [ diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml index 2b8a9063b4ad3..53a9b1937b68f 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml @@ -104,6 +104,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: + allow_system_index_access: true name: ".ml-state-write" - is_true: '' @@ -450,6 +451,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: + allow_system_index_access: true name: ".ml-state-write" - is_true: '' @@ -699,6 +701,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: + allow_system_index_access: true name: ".ml-state-write" - is_true: '' From d25c10dedc1a0035d059c5335232636b7ff50f29 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 27 Aug 2020 16:13:29 -0600 Subject: [PATCH 068/161] Allow system index access in GetAliases, as disallowing it causes very odd behavior --- docs/reference/indices/get-alias.asciidoc | 2 +- .../rest/action/admin/indices/RestGetAliasesAction.java | 9 ++++++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/docs/reference/indices/get-alias.asciidoc b/docs/reference/indices/get-alias.asciidoc index dcad9a7141431..19fb38fd2d721 100644 --- a/docs/reference/indices/get-alias.asciidoc +++ b/docs/reference/indices/get-alias.asciidoc @@ -53,7 +53,7 @@ Defaults to `all`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] + -- -Defaults to `false`. +Defaults to `true`. -- include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetAliasesAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetAliasesAction.java index ad732f531806e..a28697a148a21 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetAliasesAction.java @@ -72,8 +72,15 @@ public String getName() { return "get_aliases_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + // If not given an index (only an alias), resolves `_all`, which causes deprecation warnings. + return true; + } + static RestResponse buildRestResponse(boolean aliasesExplicitlyRequested, String[] requestedAliases, - ImmutableOpenMap> responseAliasMap, XContentBuilder builder) throws Exception { + ImmutableOpenMap> responseAliasMap, + XContentBuilder builder) throws Exception { final Set indicesToDisplay = new HashSet<>(); final Set returnedAliasNames = new HashSet<>(); for (final ObjectObjectCursor> cursor : responseAliasMap) { From a0a78f8634bda85860ec065602c4fd07e698bf05 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 27 Aug 2020 17:32:34 -0600 Subject: [PATCH 069/161] Allow system index access in GetDataStreamsStats, as disallowing it causes very odd behavior --- .../xpack/datastreams/rest/RestDataStreamsStatsAction.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/rest/RestDataStreamsStatsAction.java b/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/rest/RestDataStreamsStatsAction.java index f6f93fd05fbdb..13b676615064d 100644 --- a/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/rest/RestDataStreamsStatsAction.java +++ b/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/rest/RestDataStreamsStatsAction.java @@ -29,6 +29,12 @@ public List routes() { ); } + @Override + public boolean allowSystemIndexAccessByDefault() { + // Does not actually access system indices, but does resolve `_all`, which can cause confusing deprecation warnings. + return true; + } + @Override protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { DataStreamsStatsAction.Request dataStreamsStatsRequest = new DataStreamsStatsAction.Request(); From cba4fd7bdceab36f796c56f79a6d49ba518564d0 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 27 Aug 2020 17:39:05 -0600 Subject: [PATCH 070/161] Allow system index access in cat APIs, otherwise we get deprecation warnings --- .../org/elasticsearch/rest/action/cat/AbstractCatAction.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java index a3bf7ff3fbc98..fedbf038b5562 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java @@ -42,6 +42,11 @@ public abstract class AbstractCatAction extends BaseRestHandler { protected abstract Table getTableWithHeader(RestRequest request); + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { boolean helpWanted = request.paramAsBoolean("help", false); From a3980e392e15c48bbf6f328e44fc82d90d0817fc Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 27 Aug 2020 18:07:45 -0600 Subject: [PATCH 071/161] Remove allow_system flag from test that no longer needs it due to changed defaults --- .../src/test/resources/rest-api-spec/test/ml/index_layout.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml index 53a9b1937b68f..f186889466cd5 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml @@ -451,7 +451,6 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: - allow_system_index_access: true name: ".ml-state-write" - is_true: '' From 7770cb6d8d0d5466bc7e3a1b5c994bb4f0511e8c Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 27 Aug 2020 18:08:45 -0600 Subject: [PATCH 072/161] Remove allow_system flag from more tests that no longer needs it due to changed defaults --- .../src/test/resources/rest-api-spec/test/ml/index_layout.yml | 2 -- 1 file changed, 2 deletions(-) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml index f186889466cd5..2b8a9063b4ad3 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml @@ -104,7 +104,6 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: - allow_system_index_access: true name: ".ml-state-write" - is_true: '' @@ -700,7 +699,6 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: - allow_system_index_access: true name: ".ml-state-write" - is_true: '' From 56d4d058df864189f2bfd816ba727b36f0cfea59 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 31 Aug 2020 11:25:08 -0600 Subject: [PATCH 073/161] Add allow_system_index_access to docs tests where necessary Note: these should be removed and replaced with actual APIs everywhere this is necessary. --- x-pack/docs/en/watcher/managing-watches.asciidoc | 2 +- x-pack/docs/en/watcher/troubleshooting.asciidoc | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/x-pack/docs/en/watcher/managing-watches.asciidoc b/x-pack/docs/en/watcher/managing-watches.asciidoc index aa4c71a0dcdca..befd640673dac 100644 --- a/x-pack/docs/en/watcher/managing-watches.asciidoc +++ b/x-pack/docs/en/watcher/managing-watches.asciidoc @@ -28,7 +28,7 @@ For example, the following returns the first 100 watches: [source,console] -------------------------------------------------- -GET .watches/_search +GET .watches/_search?allow_system_index_access=true { "size" : 100 } diff --git a/x-pack/docs/en/watcher/troubleshooting.asciidoc b/x-pack/docs/en/watcher/troubleshooting.asciidoc index e6c193896f4a0..1f99bd5388a17 100644 --- a/x-pack/docs/en/watcher/troubleshooting.asciidoc +++ b/x-pack/docs/en/watcher/troubleshooting.asciidoc @@ -16,7 +16,7 @@ do that by submitting the following request: [source,console] -------------------------------------------------- -GET .watches/_mapping +GET .watches/_mapping?allow_system_index_access=true -------------------------------------------------- // TEST[setup:my_active_watch] @@ -31,7 +31,7 @@ mappings: -- [source,console] -------------------------------------------------- -DELETE .watches +DELETE .watches?allow_system_index_access=true -------------------------------------------------- // TEST[skip:index deletion] -- @@ -62,4 +62,4 @@ Keep in mind that there's no built-in validation of scripts that you add to a watch. Buggy or deliberately malicious scripts can negatively impact {watcher} performance. For example, if you add multiple watches with buggy script conditions in a short period of time, {watcher} might be temporarily unable to -process watches until the bad watches time out. \ No newline at end of file +process watches until the bad watches time out. From 15ef7e801516a2ec344f240c7b68a4fdbaf7b17b Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 31 Aug 2020 17:32:21 -0600 Subject: [PATCH 074/161] Fix origin setting in Enrich classes --- .../metadata/IndexNameExpressionResolver.java | 16 ++++++ .../xpack/enrich/EnrichPolicyRunner.java | 52 ++++++++++++------- .../TransportDeleteEnrichPolicyAction.java | 7 ++- .../TransportExecuteEnrichPolicyAction.java | 5 +- 4 files changed, 55 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index f238a1819abc7..aeb8a9722e4ba 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -92,6 +92,14 @@ public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { return concreteIndexNames(context, request.indices()); } + /** + * Same as {@link #concreteIndexNames(ClusterState, IndicesRequest)}, but access to system indices is always allowed. + */ + public String[] concreteIndexNamesWithSystemIndexAccess(ClusterState state, IndicesRequest request) { + Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), true); + return concreteIndexNames(context, request.indices()); + } + /** * Same as {@link #concreteIndices(ClusterState, IndicesOptions, String...)}, but the index expressions and options * are encapsulated in the specified request and resolves data streams. @@ -102,6 +110,14 @@ public Index[] concreteIndices(ClusterState state, IndicesRequest request) { return concreteIndices(context, request.indices()); } + /** + * Same as {@link #concreteIndices(ClusterState, IndicesRequest)}, but access to system indices is always allowed. + */ + public Index[] concreteIndicesWithSystemIndexAccess(ClusterState state, IndicesRequest request) { + Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), true); + return concreteIndices(context, request.indices()); + } + /** * Translates the provided index expression into actual concrete indices, properly deduplicated. * diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java index c3dc5b1c349d3..978b79a8e04b8 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java @@ -29,6 +29,7 @@ import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.Client; +import org.elasticsearch.client.OriginSettingClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -59,6 +60,8 @@ import java.util.Set; import java.util.function.LongSupplier; +import static org.elasticsearch.xpack.core.ClientHelper.ENRICH_ORIGIN; + public class EnrichPolicyRunner implements Runnable { private static final Logger logger = LogManager.getLogger(EnrichPolicyRunner.class); @@ -113,6 +116,7 @@ public void run() { final String[] sourceIndices = policy.getIndices().toArray(new String[0]); logger.debug("Policy [{}]: Checking source indices [{}]", policyName, sourceIndices); GetIndexRequest getIndexRequest = new GetIndexRequest().indices(sourceIndices); + // This call does not set the origin to ensure that the user executing the policy has permission to access the source index client.admin().indices().getIndex(getIndexRequest, new ActionListener<>() { @Override public void onResponse(GetIndexResponse getIndexResponse) { @@ -297,7 +301,7 @@ private void prepareAndCreateEnrichIndex() { CreateIndexRequest createEnrichIndexRequest = new CreateIndexRequest(enrichIndexName, enrichIndexSettings); createEnrichIndexRequest.mapping(resolveEnrichMapping(policy)); logger.debug("Policy [{}]: Creating new enrich index [{}]", policyName, enrichIndexName); - client.admin().indices().create(createEnrichIndexRequest, new ActionListener<>() { + enrichOriginClient().admin().indices().create(createEnrichIndexRequest, new ActionListener<>() { @Override public void onResponse(CreateIndexResponse createIndexResponse) { prepareReindexOperation(enrichIndexName); @@ -313,7 +317,7 @@ public void onFailure(Exception e) { private void prepareReindexOperation(final String destinationIndexName) { // Check to make sure that the enrich pipeline exists, and create it if it is missing. if (EnrichPolicyReindexPipeline.exists(clusterService.state()) == false) { - EnrichPolicyReindexPipeline.create(client, new ActionListener<>() { + EnrichPolicyReindexPipeline.create(enrichOriginClient(), new ActionListener<>() { @Override public void onResponse(AcknowledgedResponse acknowledgedResponse) { transferDataToEnrichIndex(destinationIndexName); @@ -347,7 +351,7 @@ private void transferDataToEnrichIndex(final String destinationIndexName) { reindexRequest.getDestination().source(new BytesArray(new byte[0]), XContentType.SMILE); reindexRequest.getDestination().routing("discard"); reindexRequest.getDestination().setPipeline(EnrichPolicyReindexPipeline.pipelineName()); - client.execute(ReindexAction.INSTANCE, reindexRequest, new ActionListener<>() { + enrichOriginClient().execute(ReindexAction.INSTANCE, reindexRequest, new ActionListener<>() { @Override public void onResponse(BulkByScrollResponse bulkByScrollResponse) { // Do we want to fail the request if there were failures during the reindex process? @@ -381,22 +385,24 @@ private void forceMergeEnrichIndex(final String destinationIndexName, final int attempt, maxForceMergeAttempts ); - client.admin().indices().forceMerge(new ForceMergeRequest(destinationIndexName).maxNumSegments(1), new ActionListener<>() { - @Override - public void onResponse(ForceMergeResponse forceMergeResponse) { - refreshEnrichIndex(destinationIndexName, attempt); - } + enrichOriginClient().admin() + .indices() + .forceMerge(new ForceMergeRequest(destinationIndexName).maxNumSegments(1), new ActionListener<>() { + @Override + public void onResponse(ForceMergeResponse forceMergeResponse) { + refreshEnrichIndex(destinationIndexName, attempt); + } - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }); + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); } private void refreshEnrichIndex(final String destinationIndexName, final int attempt) { logger.debug("Policy [{}]: Refreshing enrich index [{}]", policyName, destinationIndexName); - client.admin().indices().refresh(new RefreshRequest(destinationIndexName), new ActionListener<>() { + enrichOriginClient().admin().indices().refresh(new RefreshRequest(destinationIndexName), new ActionListener<>() { @Override public void onResponse(RefreshResponse refreshResponse) { ensureSingleSegment(destinationIndexName, attempt); @@ -410,7 +416,7 @@ public void onFailure(Exception e) { } protected void ensureSingleSegment(final String destinationIndexName, final int attempt) { - client.admin().indices().segments(new IndicesSegmentsRequest(destinationIndexName), new ActionListener<>() { + enrichOriginClient().admin().indices().segments(new IndicesSegmentsRequest(destinationIndexName), new ActionListener<>() { @Override public void onResponse(IndicesSegmentResponse indicesSegmentResponse) { IndexSegments indexSegments = indicesSegmentResponse.getIndices().get(destinationIndexName); @@ -463,7 +469,7 @@ private void setIndexReadOnly(final String destinationIndexName) { logger.debug("Policy [{}]: Setting new enrich index [{}] to be read only", policyName, destinationIndexName); UpdateSettingsRequest request = new UpdateSettingsRequest(destinationIndexName).setPreserveExisting(true) .settings(Settings.builder().put("index.auto_expand_replicas", "0-all").put("index.blocks.write", "true")); - client.admin().indices().updateSettings(request, new ActionListener<>() { + enrichOriginClient().admin().indices().updateSettings(request, new ActionListener<>() { @Override public void onResponse(AcknowledgedResponse acknowledgedResponse) { waitForIndexGreen(destinationIndexName); @@ -478,7 +484,7 @@ public void onFailure(Exception e) { private void waitForIndexGreen(final String destinationIndexName) { ClusterHealthRequest request = new ClusterHealthRequest(destinationIndexName).waitForGreenStatus(); - client.admin().cluster().health(request, new ActionListener<>() { + enrichOriginClient().admin().cluster().health(request, new ActionListener<>() { @Override public void onResponse(ClusterHealthResponse clusterHealthResponse) { updateEnrichPolicyAlias(destinationIndexName); @@ -496,7 +502,7 @@ private void updateEnrichPolicyAlias(final String destinationIndexName) { logger.debug("Policy [{}]: Promoting new enrich index [{}] to alias [{}]", policyName, destinationIndexName, enrichIndexBase); GetAliasesRequest aliasRequest = new GetAliasesRequest(enrichIndexBase); ClusterState clusterState = clusterService.state(); - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(clusterState, aliasRequest); + String[] concreteIndices = indexNameExpressionResolver.concreteIndexNamesWithSystemIndexAccess(clusterState, aliasRequest); ImmutableOpenMap> aliases = clusterState.metadata().findAliases(aliasRequest, concreteIndices); IndicesAliasesRequest aliasToggleRequest = new IndicesAliasesRequest(); String[] indices = aliases.keys().toArray(String.class); @@ -504,7 +510,7 @@ private void updateEnrichPolicyAlias(final String destinationIndexName) { aliasToggleRequest.addAliasAction(IndicesAliasesRequest.AliasActions.remove().indices(indices).alias(enrichIndexBase)); } aliasToggleRequest.addAliasAction(IndicesAliasesRequest.AliasActions.add().index(destinationIndexName).alias(enrichIndexBase)); - client.admin().indices().aliases(aliasToggleRequest, new ActionListener<>() { + enrichOriginClient().admin().indices().aliases(aliasToggleRequest, new ActionListener<>() { @Override public void onResponse(AcknowledgedResponse acknowledgedResponse) { logger.info("Policy [{}]: Policy execution complete", policyName); @@ -519,4 +525,12 @@ public void onFailure(Exception e) { } }); } + + /** + * Use this client to access information at the access level of the Enrich plugin, rather than at the access level of the user. + * For example, use this client to access system indices (such as `.enrich*` indices). + */ + private Client enrichOriginClient() { + return new OriginSettingClient(client, ENRICH_ORIGIN); + } } diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportDeleteEnrichPolicyAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportDeleteEnrichPolicyAction.java index cd72e0f07428a..d37d1cb603787 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportDeleteEnrichPolicyAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportDeleteEnrichPolicyAction.java @@ -15,6 +15,7 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.Client; +import org.elasticsearch.client.OriginSettingClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; @@ -38,6 +39,8 @@ import java.util.ArrayList; import java.util.List; +import static org.elasticsearch.xpack.core.ClientHelper.ENRICH_ORIGIN; + public class TransportDeleteEnrichPolicyAction extends TransportMasterNodeAction { private final EnrichPolicyLocks enrichPolicyLocks; @@ -132,7 +135,7 @@ protected void masterOperation( GetIndexRequest indices = new GetIndexRequest().indices(EnrichPolicy.getBaseName(request.getName()) + "-*") .indicesOptions(IndicesOptions.lenientExpand()); - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, indices); + String[] concreteIndices = indexNameExpressionResolver.concreteIndexNamesWithSystemIndexAccess(state, indices); deleteIndicesAndPolicy(concreteIndices, request.getName(), ActionListener.wrap((response) -> { enrichPolicyLocks.releasePolicy(request.getName()); @@ -153,7 +156,7 @@ private void deleteIndicesAndPolicy(String[] indices, String name, ActionListene // as the setting 'action.destructive_requires_name' may be set to true DeleteIndexRequest deleteRequest = new DeleteIndexRequest().indices(indices).indicesOptions(LENIENT_OPTIONS); - client.admin().indices().delete(deleteRequest, ActionListener.wrap((response) -> { + new OriginSettingClient(client, ENRICH_ORIGIN).admin().indices().delete(deleteRequest, ActionListener.wrap((response) -> { if (response.isAcknowledged() == false) { listener.onFailure( new ElasticsearchStatusException( diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportExecuteEnrichPolicyAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportExecuteEnrichPolicyAction.java index df3551fb4bfbf..d567290dc906d 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportExecuteEnrichPolicyAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/TransportExecuteEnrichPolicyAction.java @@ -9,7 +9,6 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.Client; -import org.elasticsearch.client.OriginSettingClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; @@ -30,8 +29,6 @@ import java.io.IOException; -import static org.elasticsearch.xpack.core.ClientHelper.ENRICH_ORIGIN; - public class TransportExecuteEnrichPolicyAction extends TransportMasterNodeAction< ExecuteEnrichPolicyAction.Request, ExecuteEnrichPolicyAction.Response> { @@ -61,7 +58,7 @@ public TransportExecuteEnrichPolicyAction( this.executor = new EnrichPolicyExecutor( settings, clusterService, - new OriginSettingClient(client, ENRICH_ORIGIN), + client, transportService.getTaskManager(), threadPool, indexNameExpressionResolver, From 8e7d9c12741a8a5b3f159762550c8b1b9eba38d7 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 1 Sep 2020 13:12:25 -0600 Subject: [PATCH 075/161] Add allow_system_index_access to reindex tests as necessary --- .../resources/rest-api-spec/test/delete_by_query/80_slices.yml | 2 ++ .../resources/rest-api-spec/test/reindex/80_slices.yml | 2 ++ .../resources/rest-api-spec/test/update_by_query/70_slices.yml | 2 ++ 3 files changed, 6 insertions(+) diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml index f4a1a9805632a..550b9a3c2f767 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml @@ -154,6 +154,7 @@ indices.refresh: {} - do: search: + allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } @@ -262,6 +263,7 @@ indices.refresh: {} - do: search: + allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml index 150ec2a4be45c..6bd34b7d40629 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml @@ -163,6 +163,7 @@ indices.refresh: {} - do: search: + allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } @@ -275,6 +276,7 @@ indices.refresh: {} - do: search: + allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml index 3e8d82f13d36c..05cf20e4253a8 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml @@ -146,6 +146,7 @@ indices.refresh: {} - do: search: + allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } @@ -249,6 +250,7 @@ indices.refresh: {} - do: search: + allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } From 712f1ccc9f9fdf9c3eda1d78490f44be9a8d80c8 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 1 Sep 2020 13:25:11 -0600 Subject: [PATCH 076/161] Add allow_system_index_access to Watcher tests & test cleanup as necessary --- .../elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java | 1 + .../rest-api-spec/test/watcher/delete_watch/10_basic.yml | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/watcher/qa/rest/src/test/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java b/x-pack/plugin/watcher/qa/rest/src/test/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java index 7dd064f7dbc91..371e6d88bc19c 100644 --- a/x-pack/plugin/watcher/qa/rest/src/test/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java +++ b/x-pack/plugin/watcher/qa/rest/src/test/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java @@ -97,6 +97,7 @@ public final void stopWatcher() throws Exception { private static void deleteWatcherIndices() throws IOException { Request deleteWatchesIndexRequest = new Request("DELETE", ".watches"); deleteWatchesIndexRequest.addParameter("ignore_unavailable", "true"); + deleteWatchesIndexRequest.addParameter("allow_system_index_access", "true"); adminClient().performRequest(deleteWatchesIndexRequest); Request deleteWatchHistoryRequest = new Request("DELETE", ".watcher-history-*"); diff --git a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/delete_watch/10_basic.yml b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/delete_watch/10_basic.yml index 1e9526ab209fa..032ea714425ec 100644 --- a/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/delete_watch/10_basic.yml +++ b/x-pack/plugin/watcher/qa/rest/src/test/resources/rest-api-spec/test/watcher/delete_watch/10_basic.yml @@ -16,7 +16,7 @@ teardown: - do: watcher.put_watch: id: "my_watch" - body: > + body: > { "trigger": { "schedule": { @@ -54,6 +54,7 @@ teardown: - do: search: rest_total_hits_as_int: true + allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 0 } From 37dc6dfb3f4eb5e87c0df28806ed29d93de96a30 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 1 Sep 2020 13:34:25 -0600 Subject: [PATCH 077/161] Add allow_system_index_access to Monitoring test cleanup as necessary --- .../smoketest/MonitoringWithWatcherRestIT.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/watcher/qa/with-monitoring/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java b/x-pack/plugin/watcher/qa/with-monitoring/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java index 34a29adac02cb..15e52ee8742e4 100644 --- a/x-pack/plugin/watcher/qa/with-monitoring/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java +++ b/x-pack/plugin/watcher/qa/with-monitoring/src/test/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java @@ -33,13 +33,15 @@ public class MonitoringWithWatcherRestIT extends ESRestTestCase { @After public void cleanExporters() throws Exception { - Request request = new Request("PUT", "/_cluster/settings"); - request.setJsonEntity(Strings.toString(jsonBuilder().startObject() + Request cleanupSettingsRequest = new Request("PUT", "/_cluster/settings"); + cleanupSettingsRequest.setJsonEntity(Strings.toString(jsonBuilder().startObject() .startObject("transient") .nullField("xpack.monitoring.exporters.*") .endObject().endObject())); - adminClient().performRequest(request); - adminClient().performRequest(new Request("DELETE", "/.watch*")); + adminClient().performRequest(cleanupSettingsRequest); + final Request deleteRequest = new Request("DELETE", "/.watch*"); + deleteRequest.addParameter("allow_system_index_access", "true"); + adminClient().performRequest(deleteRequest); } public void testThatLocalExporterAddsWatches() throws Exception { From ef3e9cdeff295786d41f0e37ec1a059aa0b770e8 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 1 Sep 2020 13:41:08 -0600 Subject: [PATCH 078/161] Add allow_system_index_access to Transform test cleanup as necessary --- .../xpack/transform/integration/TransformRestTestCase.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformRestTestCase.java b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformRestTestCase.java index 6e94e3225a6e1..da98d37e8d766 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformRestTestCase.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/transform/integration/TransformRestTestCase.java @@ -480,6 +480,7 @@ public void wipeTransforms() throws IOException { // the configuration index should be empty Request request = new Request("GET", TransformInternalIndexConstants.LATEST_INDEX_NAME + "/_search"); + request.addParameter("allow_system_index_access", "true"); try { Response searchResponse = adminClient().performRequest(request); Map searchResult = entityAsMap(searchResponse); From 83c2c52acf275420279caeaec6efbb5b09ab9b09 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 8 Sep 2020 10:42:41 -0600 Subject: [PATCH 079/161] add allow_system_index_access to Watcher cleanup --- .../org/elasticsearch/xpack/watcher/WatcherRestTestCase.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherRestTestCase.java b/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherRestTestCase.java index 42c6a4b6ecb81..07eee59af7eb8 100644 --- a/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherRestTestCase.java +++ b/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherRestTestCase.java @@ -73,6 +73,7 @@ public final void stopWatcher() throws Exception { Request deleteWatchesIndexRequest = new Request("DELETE", ".watches"); deleteWatchesIndexRequest.addParameter("ignore_unavailable", "true"); + deleteWatchesIndexRequest.addParameter("allow_system_index_access", "true"); ESRestTestCase.adminClient().performRequest(deleteWatchesIndexRequest); Request deleteWatchHistoryRequest = new Request("DELETE", ".watcher-history-*"); From 555910c7e6fcb27c7c130fe465ca7425234b3bc6 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 8 Sep 2020 14:15:06 -0600 Subject: [PATCH 080/161] Remove now unnecessary allow_system_index_access from upgrade test --- .../java/org/elasticsearch/upgrades/FullClusterRestartIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index c0b1afa807f6e..c08acfdc8e97f 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -1475,7 +1475,6 @@ public void testSystemIndexMetadataIsUpgraded() throws Exception { if (tasksCreatedVersion.before(SYSTEM_INDEX_ENFORCEMENT_VERSION)) { // Verify that the alias survived the upgrade Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); - getAliasRequest.addParameter("allow_system_index_access", "true"); Map aliasResponse = entityAsMap(client().performRequest(getAliasRequest)); assertThat(aliasResponse, hasKey(".tasks")); assertThat(aliasResponse, hasKey("test_index_reindex")); From 5e484f13b030bfa451dacb67fe86167e2b66c64a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 8 Sep 2020 14:17:45 -0600 Subject: [PATCH 081/161] Add allow_system_index_access to ML tests as necessary --- .../xpack/ml/integration/InferenceIngestIT.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java index e418296178bca..9255133ce0780 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java @@ -36,8 +36,6 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -import static org.elasticsearch.xpack.core.ml.inference.trainedmodel.inference.InferenceDefinitionTests.getClassificationDefinition; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; import static org.hamcrest.CoreMatchers.containsString; /** @@ -71,8 +69,12 @@ protected Settings restClientSettings() { @After public void cleanUpData() throws Exception { new MlRestTestStateCleaner(logger, adminClient()).clearMlMetadata(); - client().performRequest(new Request("DELETE", InferenceIndexConstants.INDEX_PATTERN)); - client().performRequest(new Request("DELETE", MlStatsIndex.indexPattern())); + final Request deleteInferenceRequest = new Request("DELETE", InferenceIndexConstants.INDEX_PATTERN); + deleteInferenceRequest.addParameter("allow_system_index_access", "true"); + client().performRequest(deleteInferenceRequest); + final Request deleteStatsRequest = new Request("DELETE", MlStatsIndex.indexPattern()); + deleteStatsRequest.addParameter("allow_system_index_access", "true"); + client().performRequest(deleteStatsRequest); Request loggingSettings = new Request("PUT", "_cluster/settings"); loggingSettings.setJsonEntity("" + "{" + From 10c3886c84254d3882b58ea813c60b78c9abaced Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 8 Sep 2020 14:30:55 -0600 Subject: [PATCH 082/161] Another allow_system_index_access test fix --- .../java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java index 95795c5ef4d0c..076406cd04584 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java @@ -119,7 +119,6 @@ public void testSystemIndicesUpgrades() throws Exception { if (tasksCreatedVersion.before(SYSTEM_INDEX_ENFORCEMENT_VERSION)) { // Verify that the alias survived the upgrade Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); - getAliasRequest.addParameter("allow_system_index_access", "true"); Map aliasResponse = entityAsMap(client().performRequest(getAliasRequest)); assertThat(aliasResponse, hasKey(".tasks")); assertThat(aliasResponse, hasKey("test_index_reindex")); From d20847bc469a8cf1b7b7a57158887d817fe4c56d Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 8 Sep 2020 15:03:36 -0600 Subject: [PATCH 083/161] Remove allow_system_index_access from TransformSurvivesUpgradeIT --- .../org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java index 2bf0d7a670495..a2cc6b746d984 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TransformSurvivesUpgradeIT.java @@ -256,7 +256,6 @@ private void awaitWrittenIndexerState(String id, Consumer> responseAss TRANSFORM_INTERNAL_INDEX_PREFIX + "*," + TRANSFORM_INTERNAL_INDEX_PREFIX_DEPRECATED + "*" + "/_search"); - getStatsDocsRequest.addParameter("allow_system_index_access", "true"); getStatsDocsRequest.setJsonEntity("{\n" + " \"query\": {\n" + From 1c579bc9ac2b2446cd39f2be0f33bbb874f0c269 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 8 Sep 2020 15:22:52 -0600 Subject: [PATCH 084/161] Add allow_system_index_access as necessary in MlMappingsUpgradeIT --- .../elasticsearch/upgrades/AbstractUpgradeTestCase.java | 9 +++++++-- .../org/elasticsearch/upgrades/MlMappingsUpgradeIT.java | 9 +++++---- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/AbstractUpgradeTestCase.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/AbstractUpgradeTestCase.java index 9041715ab474c..377204cd8f78c 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/AbstractUpgradeTestCase.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/AbstractUpgradeTestCase.java @@ -156,13 +156,15 @@ public void setupForTests() throws Exception { * index does not cause an error * @param exceptions List of keys to ignore in the index mappings. * The key is a '.' separated path. + * @param setAllowSystemIndexParam Whether the flag to allow access to system indices should be set. * @throws IOException Yes */ @SuppressWarnings("unchecked") protected void assertLegacyTemplateMatchesIndexMappings(String templateName, String indexName, boolean notAnErrorIfIndexDoesNotExist, - Set exceptions) throws IOException { + Set exceptions, + boolean setAllowSystemIndexParam) throws IOException { Request getTemplate = new Request("GET", "_template/" + templateName); Response templateResponse = client().performRequest(getTemplate); @@ -173,6 +175,9 @@ protected void assertLegacyTemplateMatchesIndexMappings(String templateName, assertNotNull(templateMappings); Request getIndexMapping = new Request("GET", indexName + "/_mapping"); + if (setAllowSystemIndexParam) { + getIndexMapping.addParameter("allow_system_index_access", "true"); + } Response indexMappingResponse; try { indexMappingResponse = client().performRequest(getIndexMapping); @@ -269,7 +274,7 @@ protected void assertLegacyTemplateMatchesIndexMappings(String templateName, protected void assertLegacyTemplateMatchesIndexMappings(String templateName, String indexName) throws IOException { - assertLegacyTemplateMatchesIndexMappings(templateName, indexName, false, Collections.emptySet()); + assertLegacyTemplateMatchesIndexMappings(templateName, indexName, false, Collections.emptySet(), false); } private boolean areBooleanObjectsAndEqual(Object a, Object b) { diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java index e35d699ee168e..687976286b50c 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java @@ -132,6 +132,7 @@ private void assertUpgradedConfigMappings() throws Exception { assertBusy(() -> { Request getMappings = new Request("GET", ".ml-config/_mappings"); + getMappings.addParameter("allow_system_index_access", "true"); Response response = client().performRequest(getMappings); Map responseLevel = entityAsMap(response); @@ -174,13 +175,13 @@ private void assertMappingsMatchTemplates() throws IOException { configIndexExceptions.add("properties.deleting.type"); configIndexExceptions.add("properties.model_memory_limit.type"); - assertLegacyTemplateMatchesIndexMappings(".ml-config", ".ml-config", false, configIndexExceptions); + assertLegacyTemplateMatchesIndexMappings(".ml-config", ".ml-config", false, configIndexExceptions, true); // the true parameter means the index may not have been created - assertLegacyTemplateMatchesIndexMappings(".ml-meta", ".ml-meta", true, Collections.emptySet()); - assertLegacyTemplateMatchesIndexMappings(".ml-stats", ".ml-stats-000001", true, Collections.emptySet()); + assertLegacyTemplateMatchesIndexMappings(".ml-meta", ".ml-meta", true, Collections.emptySet(), true); + assertLegacyTemplateMatchesIndexMappings(".ml-stats", ".ml-stats-000001", true, Collections.emptySet(), false); assertLegacyTemplateMatchesIndexMappings(".ml-state", ".ml-state-000001"); assertLegacyTemplateMatchesIndexMappings(".ml-notifications-000001", ".ml-notifications-000001"); - assertLegacyTemplateMatchesIndexMappings(".ml-inference-000003", ".ml-inference-000003", true, Collections.emptySet()); + assertLegacyTemplateMatchesIndexMappings(".ml-inference-000003", ".ml-inference-000003", true, Collections.emptySet(), true); // .ml-annotations-6 does not use a template // .ml-anomalies-shared uses a template but will have dynamically updated mappings as new jobs are opened } From bd4caf0bce37df7c1c1263fba518e282bf05695a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 8 Sep 2020 16:57:12 -0600 Subject: [PATCH 085/161] Ensure system index access is allowed for GetAliasesAction even if there's a header to disallow it Otherwise, we get unnecessary deprecation warnings --- .../admin/indices/alias/get/TransportGetAliasesAction.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java index 4fbf8e84e62bd..b1305ba2acf52 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java @@ -57,7 +57,7 @@ protected String executor() { @Override protected ClusterBlockException checkBlock(GetAliasesRequest request, ClusterState state) { return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, - indexNameExpressionResolver.concreteIndexNames(state, request)); + indexNameExpressionResolver.concreteIndexNamesWithSystemIndexAccess(state, request)); } @Override @@ -67,7 +67,7 @@ protected GetAliasesResponse read(StreamInput in) throws IOException { @Override protected void masterOperation(Task task, GetAliasesRequest request, ClusterState state, ActionListener listener) { - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); + String[] concreteIndices = indexNameExpressionResolver.concreteIndexNamesWithSystemIndexAccess(state, request); ImmutableOpenMap> aliases = state.metadata().findAliases(request, concreteIndices); listener.onResponse(new GetAliasesResponse(postProcess(request, concreteIndices, aliases))); } From 773272eb9698503315a482beb97d0110ff097c6a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 9 Sep 2020 15:10:04 -0600 Subject: [PATCH 086/161] Drop response headers during SQL index resolution to prevent deprecation warnings --- .../org/elasticsearch/xpack/ql/index/IndexResolver.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java index 536e6c825a266..d1d67bdbc77f4 100644 --- a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java @@ -7,7 +7,6 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; - import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest; @@ -18,6 +17,7 @@ import org.elasticsearch.action.fieldcaps.FieldCapabilities; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; +import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions.Option; import org.elasticsearch.action.support.IndicesOptions.WildcardStates; @@ -231,9 +231,12 @@ private void resolveIndices(String[] indices, String javaRegex, GetAliasesRespon indexRequest.indicesOptions(FROZEN_INDICES_OPTIONS); } + // Drop the response headers from this request, because this very easily triggers a spurious deprecation warning due to + // resolving system indices, despite them being filtered out later. client.admin().indices().getIndex(indexRequest, + new ContextPreservingActionListener<>(client.threadPool().getThreadContext().newRestorableContext(false), wrap(response -> filterResults(javaRegex, aliases, response, retrieveIndices, retrieveFrozenIndices, listener), - listener::onFailure)); + listener::onFailure))); } else { filterResults(javaRegex, aliases, null, false, false, listener); From 2beeb31cc597cbce3dd7a3b9b7083c05b2f1f457 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 10 Sep 2020 10:10:57 -0600 Subject: [PATCH 087/161] Drop response headers during Enrich reindex to prevent deprecation warnings --- .../xpack/enrich/EnrichPolicyRunner.java | 54 +++++++++++-------- 1 file changed, 31 insertions(+), 23 deletions(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java index 978b79a8e04b8..609443df360e4 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java @@ -27,6 +27,7 @@ import org.elasticsearch.action.admin.indices.segments.IndicesSegmentsRequest; import org.elasticsearch.action.admin.indices.segments.ShardSegments; import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; +import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.Client; import org.elasticsearch.client.OriginSettingClient; @@ -351,30 +352,37 @@ private void transferDataToEnrichIndex(final String destinationIndexName) { reindexRequest.getDestination().source(new BytesArray(new byte[0]), XContentType.SMILE); reindexRequest.getDestination().routing("discard"); reindexRequest.getDestination().setPipeline(EnrichPolicyReindexPipeline.pipelineName()); - enrichOriginClient().execute(ReindexAction.INSTANCE, reindexRequest, new ActionListener<>() { - @Override - public void onResponse(BulkByScrollResponse bulkByScrollResponse) { - // Do we want to fail the request if there were failures during the reindex process? - if (bulkByScrollResponse.getBulkFailures().size() > 0) { - listener.onFailure(new ElasticsearchException("Encountered bulk failures during reindex process")); - } else if (bulkByScrollResponse.getSearchFailures().size() > 0) { - listener.onFailure(new ElasticsearchException("Encountered search failures during reindex process")); - } else { - logger.info( - "Policy [{}]: Transferred [{}] documents to enrich index [{}]", - policyName, - bulkByScrollResponse.getCreated(), - destinationIndexName - ); - forceMergeEnrichIndex(destinationIndexName, 1); - } - } + client.execute( + ReindexAction.INSTANCE, + reindexRequest, + new ContextPreservingActionListener<>( + client.threadPool().getThreadContext().newRestorableContext(false), + new ActionListener<>() { + @Override + public void onResponse(BulkByScrollResponse bulkByScrollResponse) { + // Do we want to fail the request if there were failures during the reindex process? + if (bulkByScrollResponse.getBulkFailures().size() > 0) { + listener.onFailure(new ElasticsearchException("Encountered bulk failures during reindex process")); + } else if (bulkByScrollResponse.getSearchFailures().size() > 0) { + listener.onFailure(new ElasticsearchException("Encountered search failures during reindex process")); + } else { + logger.info( + "Policy [{}]: Transferred [{}] documents to enrich index [{}]", + policyName, + bulkByScrollResponse.getCreated(), + destinationIndexName + ); + forceMergeEnrichIndex(destinationIndexName, 1); + } + } - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }); + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + } + ) + ); } private void forceMergeEnrichIndex(final String destinationIndexName, final int attempt) { From 646dd12fd9bd1d31b3168f253caaefd622edd795 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 10 Sep 2020 11:26:21 -0600 Subject: [PATCH 088/161] Fix compilation error due to merge --- .../action/DeleteDataStreamTransportActionTests.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/action/DeleteDataStreamTransportActionTests.java b/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/action/DeleteDataStreamTransportActionTests.java index 66f4d9acbc2b4..d51a8137f33fa 100644 --- a/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/action/DeleteDataStreamTransportActionTests.java +++ b/x-pack/plugin/data-streams/src/test/java/org/elasticsearch/xpack/datastreams/action/DeleteDataStreamTransportActionTests.java @@ -17,6 +17,8 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.Index; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; @@ -37,7 +39,7 @@ public class DeleteDataStreamTransportActionTests extends ESTestCase { - private final IndexNameExpressionResolver iner = new IndexNameExpressionResolver(); + private final IndexNameExpressionResolver iner = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)); public void testDeleteDataStream() { final String dataStreamName = "my-data-stream"; From 02cf28e5a45d340219d8701342502fd3586895ec Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 10 Sep 2020 16:16:19 -0600 Subject: [PATCH 089/161] ThreadPool -> ThreadContext in test per review --- .../cluster/ClusterModuleTests.java | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java b/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java index 9e8afca134329..2f81d695fe99d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java @@ -49,11 +49,10 @@ import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsModule; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.plugins.ClusterPlugin; import org.elasticsearch.test.gateway.TestGatewayAllocator; -import org.elasticsearch.threadpool.TestThreadPool; -import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; import java.util.Collection; @@ -66,12 +65,12 @@ public class ClusterModuleTests extends ModuleTestCase { private ClusterInfoService clusterInfoService = EmptyClusterInfoService.INSTANCE; private ClusterService clusterService; - private ThreadPool threadPool; + private ThreadContext threadContext; @Override public void setUp() throws Exception { super.setUp(); - threadPool = new TestThreadPool(this.getClass().getSimpleName() + "ThreadPool"); + threadContext = new ThreadContext(Settings.EMPTY); clusterService = new ClusterService(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null); } @@ -79,7 +78,6 @@ public void setUp() throws Exception { @Override public void tearDown() throws Exception { super.tearDown(); - threadPool.shutdown(); clusterService.close(); } @@ -137,7 +135,7 @@ public void testRegisterAllocationDeciderDuplicate() { public Collection createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) { return Collections.singletonList(new EnableAllocationDecider(settings, clusterSettings)); } - }), clusterInfoService, threadPool.getThreadContext())); + }), clusterInfoService, threadContext)); assertEquals(e.getMessage(), "Cannot specify allocation decider [" + EnableAllocationDecider.class.getName() + "] twice"); } @@ -149,7 +147,7 @@ public void testRegisterAllocationDecider() { public Collection createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) { return Collections.singletonList(new FakeAllocationDecider()); } - }), clusterInfoService, threadPool.getThreadContext()); + }), clusterInfoService, threadContext); assertTrue(module.deciderList.stream().anyMatch(d -> d.getClass().equals(FakeAllocationDecider.class))); } @@ -161,7 +159,7 @@ public Map> getShardsAllocators(Settings setti return Collections.singletonMap(name, supplier); } } - ), clusterInfoService, threadPool.getThreadContext()); + ), clusterInfoService, threadContext); } public void testRegisterShardsAllocator() { @@ -179,7 +177,7 @@ public void testRegisterShardsAllocatorAlreadyRegistered() { public void testUnknownShardsAllocator() { Settings settings = Settings.builder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING.getKey(), "dne").build(); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> - new ClusterModule(settings, clusterService, Collections.emptyList(), clusterInfoService, threadPool.getThreadContext())); + new ClusterModule(settings, clusterService, Collections.emptyList(), clusterInfoService, threadContext)); assertEquals("Unknown ShardsAllocator [dne]", e.getMessage()); } @@ -222,14 +220,14 @@ public void testAllocationDeciderOrder() { public void testRejectsReservedExistingShardsAllocatorName() { final ClusterModule clusterModule = new ClusterModule(Settings.EMPTY, clusterService, - List.of(existingShardsAllocatorPlugin(GatewayAllocator.ALLOCATOR_NAME)), clusterInfoService, threadPool.getThreadContext()); + List.of(existingShardsAllocatorPlugin(GatewayAllocator.ALLOCATOR_NAME)), clusterInfoService, threadContext); expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator())); } public void testRejectsDuplicateExistingShardsAllocatorName() { final ClusterModule clusterModule = new ClusterModule(Settings.EMPTY, clusterService, List.of(existingShardsAllocatorPlugin("duplicate"), existingShardsAllocatorPlugin("duplicate")), - clusterInfoService, threadPool.getThreadContext()); + clusterInfoService, threadContext); expectThrows(IllegalArgumentException.class, () -> clusterModule.setExistingShardsAllocators(new TestGatewayAllocator())); } From c405757112765add8ec4bd096980c6840377a4cd Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 10 Sep 2020 16:19:53 -0600 Subject: [PATCH 090/161] Formatting per review --- .../cluster/metadata/DateMathExpressionResolverTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java index 35b113b1c751f..527c6618f6bc8 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/DateMathExpressionResolverTests.java @@ -43,7 +43,7 @@ public class DateMathExpressionResolverTests extends ESTestCase { private final DateMathExpressionResolver expressionResolver = new DateMathExpressionResolver(); private final Context context = new Context( - ClusterState.builder(new ClusterName("_name")).build(), IndicesOptions.strictExpand(), + ClusterState.builder(new ClusterName("_name")).build(), IndicesOptions.strictExpand(), false ); From 0be7d2b9c1d0746f8fddd0402cd7535e40ad1ee9 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 10 Sep 2020 16:25:57 -0600 Subject: [PATCH 091/161] Add comment on enrich hack --- .../org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java index 609443df360e4..8d771c0266fc0 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/EnrichPolicyRunner.java @@ -352,6 +352,12 @@ private void transferDataToEnrichIndex(final String destinationIndexName) { reindexRequest.getDestination().source(new BytesArray(new byte[0]), XContentType.SMILE); reindexRequest.getDestination().routing("discard"); reindexRequest.getDestination().setPipeline(EnrichPolicyReindexPipeline.pipelineName()); + + // The ContextPreservingActionListener here is for the purpose of dropping the response headers, as we need this reindex to run + // in the security context of the user (rather than Enrich's security context) to ensure that DLS/FLS is correctly applied, but + // the reindex needs to access the `.enrich` index, which causes a deprecation warning. Since we drop the response headers, + // the deprecation warning is also dropped - but this is a hack and will not work once full protections of system indices are + // enabled. client.execute( ReindexAction.INSTANCE, reindexRequest, From 828879c4e0a3892f7bddfb683f744eace2646992 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 14 Sep 2020 11:33:16 -0600 Subject: [PATCH 092/161] Expect deprecation warnings in ML upgrade tests --- ...nfigIndexMappingsFullClusterRestartIT.java | 16 +++++++-- .../upgrades/MlMappingsUpgradeIT.java | 2 +- .../rest/IndexMappingTemplateAsserter.java | 34 +++++++++++++------ 3 files changed, 37 insertions(+), 15 deletions(-) diff --git a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java index ca4ba53e36aaf..2796a1a9e49e4 100644 --- a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java +++ b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java @@ -65,13 +65,18 @@ public void testMlConfigIndexMappingsAfterMigration() throws Exception { createAnomalyDetectorJob(NEW_CLUSTER_JOB_ID); // assert that the mappings are updated - IndexMappingTemplateAsserter.assertMlMappingsMatchTemplates(client()); + IndexMappingTemplateAsserter.assertMlMappingsMatchTemplates(client(), true); } } private void assertThatMlConfigIndexDoesNotExist() { Request getIndexRequest = new Request("GET", ".ml-config"); - getIndexRequest.addParameter("allow_system_index_access", "true"); + getIndexRequest.setOptions(expectVersionSpecificWarnings(v -> { + final String systemIndexWarning = "this request accesses system indices: [.ml-config], but in a future major version, direct " + + "access to system indices will be prevented by default"; + v.current(systemIndexWarning); + v.compatible(systemIndexWarning); + })); ResponseException e = expectThrows(ResponseException.class, () -> client().performRequest(getIndexRequest)); assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(404)); } @@ -99,7 +104,12 @@ private void createAnomalyDetectorJob(String jobId) throws IOException { @SuppressWarnings("unchecked") private Map getConfigIndexMappings() throws Exception { Request getIndexMappingsRequest = new Request("GET", ".ml-config/_mappings"); - getIndexMappingsRequest.addParameter("allow_system_index_access", "true"); + getIndexMappingsRequest.setOptions(expectVersionSpecificWarnings(v -> { + final String systemIndexWarning = "this request accesses system indices: [.ml-config], but in a future major version, direct " + + "access to system indices will be prevented by default"; + v.current(systemIndexWarning); + v.compatible(systemIndexWarning); + })); Response getIndexMappingsResponse = client().performRequest(getIndexMappingsRequest); assertThat(getIndexMappingsResponse.getStatusLine().getStatusCode(), equalTo(200)); diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java index 2593eb0f5a476..e8dab738f1fd4 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java @@ -54,7 +54,7 @@ public void testMappingsUpgrade() throws Exception { assertUpgradedResultsMappings(); closeAndReopenTestJob(); assertUpgradedConfigMappings(); - IndexMappingTemplateAsserter.assertMlMappingsMatchTemplates(client()); + IndexMappingTemplateAsserter.assertMlMappingsMatchTemplates(client(), false); break; default: throw new UnsupportedOperationException("Unknown cluster type [" + CLUSTER_TYPE + "]"); diff --git a/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java index 18cf346103962..fd1b830cd61e3 100644 --- a/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java +++ b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java @@ -51,9 +51,10 @@ public class IndexMappingTemplateAsserter { * effect of a different test running in the cluster. * * @param client The rest client + * @param allowSystemIndexWarnings Whether deprecation warnings for system index access should be allowed/expected. * @throws IOException On error */ - public static void assertMlMappingsMatchTemplates(RestClient client) throws IOException { + public static void assertMlMappingsMatchTemplates(RestClient client, boolean allowSystemIndexWarnings) throws IOException { // Keys that have been dynamically mapped in the .ml-config index // but are not in the template. These can only be fixed with // re-index and should be addressed at the next major upgrade. @@ -68,13 +69,14 @@ public static void assertMlMappingsMatchTemplates(RestClient client) throws IOEx configIndexExceptions.add("properties.deleting.type"); configIndexExceptions.add("properties.model_memory_limit.type"); - assertLegacyTemplateMatchesIndexMappings(client, ".ml-config", ".ml-config", false, configIndexExceptions); + assertLegacyTemplateMatchesIndexMappings(client, ".ml-config", ".ml-config", false, configIndexExceptions, true); // the true parameter means the index may not have been created - assertLegacyTemplateMatchesIndexMappings(client, ".ml-meta", ".ml-meta", true, Collections.emptySet()); - assertLegacyTemplateMatchesIndexMappings(client, ".ml-stats", ".ml-stats-000001", true, Collections.emptySet()); - assertLegacyTemplateMatchesIndexMappings(client, ".ml-state", ".ml-state-000001", true, Collections.emptySet()); + assertLegacyTemplateMatchesIndexMappings(client, ".ml-meta", ".ml-meta", true, Collections.emptySet(), true); + assertLegacyTemplateMatchesIndexMappings(client, ".ml-stats", ".ml-stats-000001", true, Collections.emptySet(), false); + assertLegacyTemplateMatchesIndexMappings(client, ".ml-state", ".ml-state-000001", true, Collections.emptySet(), false); assertLegacyTemplateMatchesIndexMappings(client, ".ml-notifications-000001", ".ml-notifications-000001"); - assertLegacyTemplateMatchesIndexMappings(client, ".ml-inference-000003", ".ml-inference-000003", true, Collections.emptySet()); + assertLegacyTemplateMatchesIndexMappings(client, ".ml-inference-000003", ".ml-inference-000003", true, Collections.emptySet(), + true); // .ml-annotations-6 does not use a template // .ml-anomalies-shared uses a template but will have dynamically updated mappings as new jobs are opened } @@ -105,14 +107,16 @@ public static void assertMlMappingsMatchTemplates(RestClient client) throws IOEx * index does not cause an error * @param exceptions List of keys to ignore in the index mappings. * Each key is a '.' separated path. + * @param allowSystemIndexWarnings Whether deprecation warnings for system index access should be allowed/expected. * @throws IOException Yes */ @SuppressWarnings("unchecked") public static void assertLegacyTemplateMatchesIndexMappings(RestClient client, - String templateName, - String indexName, - boolean notAnErrorIfIndexDoesNotExist, - Set exceptions) throws IOException { + String templateName, + String indexName, + boolean notAnErrorIfIndexDoesNotExist, + Set exceptions, + boolean allowSystemIndexWarnings) throws IOException { Request getTemplate = new Request("GET", "_template/" + templateName); Response templateResponse = client.performRequest(getTemplate); @@ -124,6 +128,14 @@ public static void assertLegacyTemplateMatchesIndexMappings(RestClient client, assertNotNull(templateMappings); Request getIndexMapping = new Request("GET", indexName + "/_mapping"); + if (allowSystemIndexWarnings) { + final String systemIndexWarning = "this request accesses system indices: [" + indexName + "], but in a future major version, " + + "direct access to system indices will be prevented by default"; + getIndexMapping.setOptions(ESRestTestCase.expectVersionSpecificWarnings(v -> { + v.current(systemIndexWarning); + v.compatible(systemIndexWarning); + })); + } Response indexMappingResponse; try { indexMappingResponse = client.performRequest(getIndexMapping); @@ -222,7 +234,7 @@ public static void assertLegacyTemplateMatchesIndexMappings(RestClient client, public static void assertLegacyTemplateMatchesIndexMappings(RestClient client, String templateName, String indexName) throws IOException { - assertLegacyTemplateMatchesIndexMappings(client, templateName, indexName, false, Collections.emptySet()); + assertLegacyTemplateMatchesIndexMappings(client, templateName, indexName, false, Collections.emptySet(), false); } private static boolean areBooleanObjectsAndEqual(Object a, Object b) { From ba55f39fc3a618ca3e71746b59737fd4d91b65e6 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 15 Sep 2020 10:50:31 -0600 Subject: [PATCH 093/161] Wording change per review --- docs/reference/api-conventions.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index 50a34a48b9773..aaa7dd7d52db8 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -43,7 +43,7 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] The defaults settings for the above parameters depend on the API being used. Some indices (hereafter "system indices") are used by various system -components and/or plugins to store state or configuration. These indices +modules and/or plugins to store state or configuration. These indices are not intended to be accessed directly, and accessing them directly is deprecated. In the next major version, access to these indices will be prevented by default to prevent accidental operations. You can opt in to access to these From a67c2b29ab0335dfbcb7fade9de3421d585996f7 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 15 Sep 2020 10:52:57 -0600 Subject: [PATCH 094/161] Update comment per review --- .../src/test/java/org/elasticsearch/http/SystemIndexRestIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java index d479e6c2c99de..99db3b1a597b4 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -117,7 +117,7 @@ private void assertDeprecationWarningOnAccess(String queryPattern, String warnin .build(); Request searchRequest = new Request("GET", "/" + queryPattern + randomFrom("/_count", "/_search")); searchRequest.setJsonEntity("{\"query\": {\"match\": {\"some_field\": \"some_value\"}}}"); - // Disallow no indices to cause an exception if this resolves to zero indices (as we expect) + // Disallow no indices to cause an exception if this resolves to zero indices, so that we're sure it resolved the index searchRequest.addParameter("allow_no_indices", "false"); searchRequest.setOptions(expectWarningOptions); From 4c8763bc981d3e4472de45a9889acf55b6a853e7 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 15 Sep 2020 11:03:35 -0600 Subject: [PATCH 095/161] Factor out shared stream/map code per review --- .../IndexNameExpressionResolverTests.java | 47 ++++++------------- 1 file changed, 15 insertions(+), 32 deletions(-) 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 eecd346583832..87448761200f3 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -1841,10 +1841,7 @@ public void testSystemIndexResolutionWhenAllowed() { { SearchRequest request = new SearchRequest(".ml-meta"); - List indexNames = Arrays - .stream(indexNameExpressionResolver.concreteIndices(state, request)) - .map(i -> i.getName()) - .collect(Collectors.toList()); + List indexNames = getConcreteIndexNames(state, request); assertThat(indexNames, containsInAnyOrder(".ml-meta")); } @@ -1852,10 +1849,7 @@ public void testSystemIndexResolutionWhenAllowed() { { SearchRequest request = new SearchRequest(".ml-*"); - List indexNames = Arrays - .stream(indexNameExpressionResolver.concreteIndices(state, request)) - .map(i -> i.getName()) - .collect(Collectors.toList()); + List indexNames = getConcreteIndexNames(state, request); assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); } @@ -1863,10 +1857,7 @@ public void testSystemIndexResolutionWhenAllowed() { { SearchRequest request = new SearchRequest(".w*"); - List indexNames = Arrays - .stream(indexNameExpressionResolver.concreteIndices(state, request)) - .map(i -> i.getName()) - .collect(Collectors.toList()); + List indexNames = getConcreteIndexNames(state, request); assertThat(indexNames, containsInAnyOrder(".watches")); } @@ -1874,10 +1865,7 @@ public void testSystemIndexResolutionWhenAllowed() { { SearchRequest request = new SearchRequest(randomFrom("*", "_all")); - List indexNames = Arrays - .stream(indexNameExpressionResolver.concreteIndices(state, request)) - .map(i -> i.getName()) - .collect(Collectors.toList()); + List indexNames = getConcreteIndexNames(state, request); assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); } } @@ -1898,10 +1886,7 @@ public void testSystemIndexResolutionBlocked() { { SearchRequest request = new SearchRequest(".ml-*"); - List indexNames = Arrays - .stream(indexNameExpressionResolver.concreteIndices(state, request)) - .map(i -> i.getName()) - .collect(Collectors.toList()); + List indexNames = getConcreteIndexNames(state, request); assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff], but in a future major version, direct access " + "to system indices will be prevented by default"); @@ -1911,10 +1896,7 @@ public void testSystemIndexResolutionBlocked() { { SearchRequest request = new SearchRequest(".w*"); - List indexNames = Arrays - .stream(indexNameExpressionResolver.concreteIndices(state, request)) - .map(i -> i.getName()) - .collect(Collectors.toList()); + List indexNames = getConcreteIndexNames(state, request); assertThat(indexNames, containsInAnyOrder(".watches")); assertWarnings("this request accesses system indices: [.watches], but in a future major version, direct access " + "to system indices will be prevented by default"); @@ -1924,10 +1906,7 @@ public void testSystemIndexResolutionBlocked() { { SearchRequest request = new SearchRequest(".ml-meta"); - List indexNames = Arrays - .stream(indexNameExpressionResolver.concreteIndices(state, request)) - .map(i -> i.getName()) - .collect(Collectors.toList()); + List indexNames = getConcreteIndexNames(state, request); assertThat(indexNames, containsInAnyOrder(".ml-meta")); assertWarnings("this request accesses system indices: [.ml-meta], but in a future major version, direct access " + "to system indices will be prevented by default"); @@ -1937,10 +1916,7 @@ public void testSystemIndexResolutionBlocked() { { SearchRequest request = new SearchRequest(randomFrom("*", "_all")); - List indexNames = Arrays - .stream(indexNameExpressionResolver.concreteIndices(state, request)) - .map(i -> i.getName()) - .collect(Collectors.toList()); + List indexNames = getConcreteIndexNames(state, request); assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff, .watches], but in a future major version, direct " + "access to system indices will be prevented by default"); @@ -2180,4 +2156,11 @@ public void testDataStreamsNames() { names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), "*", "-*"); assertThat(names, empty()); } + + private List getConcreteIndexNames(ClusterState state, SearchRequest request) { + return Arrays + .stream(indexNameExpressionResolver.concreteIndices(state, request)) + .map(i -> i.getName()) + .collect(Collectors.toList()); + } } From fcd6af9d5f2dc488276aa9e2be09f2352babcfde Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 15 Sep 2020 11:21:01 -0600 Subject: [PATCH 096/161] Break up large INER test cases per review --- .../IndexNameExpressionResolverTests.java | 143 +++++++++--------- 1 file changed, 69 insertions(+), 74 deletions(-) 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 87448761200f3..5d2031be26f93 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -1828,99 +1828,84 @@ public void testIgnoreThrottled() { } } - public void testSystemIndexResolutionWhenAllowed() { - Settings settings = Settings.builder().build(); - Metadata.Builder mdBuilder = Metadata.builder() - .put(indexBuilder(".ml-meta", settings).state(State.OPEN).system(true)) - .put(indexBuilder(".watches", settings).state(State.OPEN).system(true)) - .put(indexBuilder(".ml-stuff", settings).state(State.OPEN).system(true)) - .put(indexBuilder("some-other-index").state(State.OPEN)); - ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); + public void testFullWildcardSystemIndexResolutionAllowed() { + ClusterState state = systemIndexTestClusterState(); + SearchRequest request = new SearchRequest(randomFrom("*", "_all")); - // Single name - { - SearchRequest request = new SearchRequest(".ml-meta"); - - List indexNames = getConcreteIndexNames(state, request); - assertThat(indexNames, containsInAnyOrder(".ml-meta")); - } + List indexNames = getConcreteIndexNames(state, request); + assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); + } - // Wildcard that should match multiple - { - SearchRequest request = new SearchRequest(".ml-*"); + public void testWildcardSystemIndexResolutionMultipleMatchesAllowed() { + ClusterState state = systemIndexTestClusterState(); + SearchRequest request = new SearchRequest(".w*"); - List indexNames = getConcreteIndexNames(state, request); - assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); - } + List indexNames = getConcreteIndexNames(state, request); + assertThat(indexNames, containsInAnyOrder(".watches")); + } - // Wildcard that just matches one - { - SearchRequest request = new SearchRequest(".w*"); + public void testWildcardSystemIndexResolutionSingleMatchAllowed() { + ClusterState state = systemIndexTestClusterState(); + SearchRequest request = new SearchRequest(".ml-*"); - List indexNames = getConcreteIndexNames(state, request); - assertThat(indexNames, containsInAnyOrder(".watches")); - } + List indexNames = getConcreteIndexNames(state, request); + assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); + } - // Full wildcard - { - SearchRequest request = new SearchRequest(randomFrom("*", "_all")); + public void testSingleSystemIndexResolutionAllowed() { + ClusterState state = systemIndexTestClusterState(); + SearchRequest request = new SearchRequest(".ml-meta"); - List indexNames = getConcreteIndexNames(state, request); - assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); - } + List indexNames = getConcreteIndexNames(state, request); + assertThat(indexNames, containsInAnyOrder(".ml-meta")); } - public void testSystemIndexResolutionBlocked() { - // Set up the thread context to disallow system index access + public void testFullWildcardSystemIndexResolutionDeprecated() { threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); + ClusterState state = systemIndexTestClusterState(); + SearchRequest request = new SearchRequest(randomFrom("*", "_all")); - Settings settings = Settings.builder().build(); - Metadata.Builder mdBuilder = Metadata.builder() - .put(indexBuilder(".ml-meta", settings).state(State.OPEN).system(true)) - .put(indexBuilder(".watches", settings).state(State.OPEN).system(true)) - .put(indexBuilder(".ml-stuff", settings).state(State.OPEN).system(true)) - .put(indexBuilder("some-other-index").state(State.OPEN)); - ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); + List indexNames = getConcreteIndexNames(state, request); + assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); + assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff, .watches], but in a future major version, " + + "direct access to system indices will be prevented by default"); - // Wildcard that might match multiple - { - SearchRequest request = new SearchRequest(".ml-*"); + } - List indexNames = getConcreteIndexNames(state, request); - assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); - assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff], but in a future major version, direct access " + - "to system indices will be prevented by default"); - } + public void testSingleSystemIndexResolutionDeprecated() { + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); + ClusterState state = systemIndexTestClusterState(); + SearchRequest request = new SearchRequest(".ml-meta"); - // Wilcard that might match a single index - { - SearchRequest request = new SearchRequest(".w*"); + List indexNames = getConcreteIndexNames(state, request); + assertThat(indexNames, containsInAnyOrder(".ml-meta")); + assertWarnings("this request accesses system indices: [.ml-meta], but in a future major version, direct access " + + "to system indices will be prevented by default"); - List indexNames = getConcreteIndexNames(state, request); - assertThat(indexNames, containsInAnyOrder(".watches")); - assertWarnings("this request accesses system indices: [.watches], but in a future major version, direct access " + - "to system indices will be prevented by default"); - } + } - // A specific index name - { - SearchRequest request = new SearchRequest(".ml-meta"); + public void testWildcardSystemIndexReslutionSingleMatchDeprecated() { + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); + ClusterState state = systemIndexTestClusterState(); + SearchRequest request = new SearchRequest(".w*"); - List indexNames = getConcreteIndexNames(state, request); - assertThat(indexNames, containsInAnyOrder(".ml-meta")); - assertWarnings("this request accesses system indices: [.ml-meta], but in a future major version, direct access " + - "to system indices will be prevented by default"); - } + List indexNames = getConcreteIndexNames(state, request); + assertThat(indexNames, containsInAnyOrder(".watches")); + assertWarnings("this request accesses system indices: [.watches], but in a future major version, direct access " + + "to system indices will be prevented by default"); - // Full wildcard - { - SearchRequest request = new SearchRequest(randomFrom("*", "_all")); + } + + public void testWildcardSystemIndexResolutionMultipleMatchesDeprecated() { + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); + ClusterState state = systemIndexTestClusterState(); + SearchRequest request = new SearchRequest(".ml-*"); + + List indexNames = getConcreteIndexNames(state, request); + assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); + assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff], but in a future major version, direct access " + + "to system indices will be prevented by default"); - List indexNames = getConcreteIndexNames(state, request); - assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); - assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff, .watches], but in a future major version, direct " + - "access to system indices will be prevented by default"); - } } public void testDataStreams() { @@ -2157,6 +2142,16 @@ public void testDataStreamsNames() { assertThat(names, empty()); } + private ClusterState systemIndexTestClusterState() { + Settings settings = Settings.builder().build(); + Metadata.Builder mdBuilder = Metadata.builder() + .put(indexBuilder(".ml-meta", settings).state(State.OPEN).system(true)) + .put(indexBuilder(".watches", settings).state(State.OPEN).system(true)) + .put(indexBuilder(".ml-stuff", settings).state(State.OPEN).system(true)) + .put(indexBuilder("some-other-index").state(State.OPEN)); + return ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); + } + private List getConcreteIndexNames(ClusterState state, SearchRequest request) { return Arrays .stream(indexNameExpressionResolver.concreteIndices(state, request)) From c3bedf6eb0cab33747b29b7a9d552b5b81ac9eb8 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 15 Sep 2020 11:32:54 -0600 Subject: [PATCH 097/161] Tweak method name --- .../IndexNameExpressionResolverTests.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) 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 5d2031be26f93..ade851cf15f9d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -1832,7 +1832,7 @@ public void testFullWildcardSystemIndexResolutionAllowed() { ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(randomFrom("*", "_all")); - List indexNames = getConcreteIndexNames(state, request); + List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); } @@ -1840,7 +1840,7 @@ public void testWildcardSystemIndexResolutionMultipleMatchesAllowed() { ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".w*"); - List indexNames = getConcreteIndexNames(state, request); + List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder(".watches")); } @@ -1848,7 +1848,7 @@ public void testWildcardSystemIndexResolutionSingleMatchAllowed() { ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".ml-*"); - List indexNames = getConcreteIndexNames(state, request); + List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); } @@ -1856,7 +1856,7 @@ public void testSingleSystemIndexResolutionAllowed() { ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".ml-meta"); - List indexNames = getConcreteIndexNames(state, request); + List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder(".ml-meta")); } @@ -1865,7 +1865,7 @@ public void testFullWildcardSystemIndexResolutionDeprecated() { ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(randomFrom("*", "_all")); - List indexNames = getConcreteIndexNames(state, request); + List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff, .watches], but in a future major version, " + "direct access to system indices will be prevented by default"); @@ -1877,7 +1877,7 @@ public void testSingleSystemIndexResolutionDeprecated() { ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".ml-meta"); - List indexNames = getConcreteIndexNames(state, request); + List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder(".ml-meta")); assertWarnings("this request accesses system indices: [.ml-meta], but in a future major version, direct access " + "to system indices will be prevented by default"); @@ -1889,7 +1889,7 @@ public void testWildcardSystemIndexReslutionSingleMatchDeprecated() { ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".w*"); - List indexNames = getConcreteIndexNames(state, request); + List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder(".watches")); assertWarnings("this request accesses system indices: [.watches], but in a future major version, direct access " + "to system indices will be prevented by default"); @@ -1901,7 +1901,7 @@ public void testWildcardSystemIndexResolutionMultipleMatchesDeprecated() { ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".ml-*"); - List indexNames = getConcreteIndexNames(state, request); + List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder(".ml-meta", ".ml-stuff")); assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff], but in a future major version, direct access " + "to system indices will be prevented by default"); @@ -2152,7 +2152,7 @@ private ClusterState systemIndexTestClusterState() { return ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build(); } - private List getConcreteIndexNames(ClusterState state, SearchRequest request) { + private List resolveConcreteIndexNameList(ClusterState state, SearchRequest request) { return Arrays .stream(indexNameExpressionResolver.concreteIndices(state, request)) .map(i -> i.getName()) From 70a8114ebaab913c1a06dc02ae152b93c5c87943 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 15 Sep 2020 11:57:09 -0600 Subject: [PATCH 098/161] Actually pass parameter in ML upgrade tests per review --- .../xpack/test/rest/IndexMappingTemplateAsserter.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java index fd1b830cd61e3..f2d130f88251f 100644 --- a/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java +++ b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java @@ -69,14 +69,16 @@ public static void assertMlMappingsMatchTemplates(RestClient client, boolean all configIndexExceptions.add("properties.deleting.type"); configIndexExceptions.add("properties.model_memory_limit.type"); - assertLegacyTemplateMatchesIndexMappings(client, ".ml-config", ".ml-config", false, configIndexExceptions, true); + assertLegacyTemplateMatchesIndexMappings(client, ".ml-config", ".ml-config", false, configIndexExceptions, + allowSystemIndexWarnings); // the true parameter means the index may not have been created - assertLegacyTemplateMatchesIndexMappings(client, ".ml-meta", ".ml-meta", true, Collections.emptySet(), true); + assertLegacyTemplateMatchesIndexMappings(client, ".ml-meta", ".ml-meta", true, Collections.emptySet(), + allowSystemIndexWarnings); assertLegacyTemplateMatchesIndexMappings(client, ".ml-stats", ".ml-stats-000001", true, Collections.emptySet(), false); assertLegacyTemplateMatchesIndexMappings(client, ".ml-state", ".ml-state-000001", true, Collections.emptySet(), false); assertLegacyTemplateMatchesIndexMappings(client, ".ml-notifications-000001", ".ml-notifications-000001"); assertLegacyTemplateMatchesIndexMappings(client, ".ml-inference-000003", ".ml-inference-000003", true, Collections.emptySet(), - true); + allowSystemIndexWarnings); // .ml-annotations-6 does not use a template // .ml-anomalies-shared uses a template but will have dynamically updated mappings as new jobs are opened } From a621a3ebf0c337bdeafa9bae597e206ac7cf68cc Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 15 Sep 2020 14:05:42 -0600 Subject: [PATCH 099/161] Break up IndexAbstractionTests per review --- .../metadata/IndexAbstractionTests.java | 132 ++++++++++-------- 1 file changed, 74 insertions(+), 58 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexAbstractionTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexAbstractionTests.java index 327de108ffe28..6c58692d305b7 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexAbstractionTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexAbstractionTests.java @@ -27,13 +27,14 @@ import org.elasticsearch.test.VersionUtils; import java.util.Objects; -import java.util.Random; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; public class IndexAbstractionTests extends ESTestCase { + public static final String SYSTEM_ALIAS_NAME = "system_alias"; + public void testHiddenAliasValidation() { final String hiddenAliasName = "hidden_alias"; AliasMetadata hiddenAliasMetadata = new AliasMetadata.Builder(hiddenAliasName).isHidden(true).build(); @@ -118,71 +119,86 @@ public void testHiddenAliasValidation() { } } - public void testSystemAliasValidation() { - final Random random = random(); - final String systemAlias = "system_alias"; - final Version random7xVersion = VersionUtils.randomVersionBetween(random, Version.V_7_0_0, + public void testSystemAliasValidationMixedVersionSystemAndRegularFails() { + final Version random7xVersion = VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, VersionUtils.getPreviousVersion(Version.V_8_0_0)); - final AliasMetadata aliasMetadata = new AliasMetadata.Builder(systemAlias).build(); - - final IndexMetadata currentVersionSystem = buildIndexWithAlias(".system1", systemAlias, null, Version.CURRENT, true); - final IndexMetadata currentVersionSystem2 = buildIndexWithAlias(".system2", systemAlias, null, Version.CURRENT, true); - final IndexMetadata oldVersionSystem = buildIndexWithAlias(".oldVersionSystem", systemAlias, null, random7xVersion, true); + final AliasMetadata aliasMetadata = new AliasMetadata.Builder(SYSTEM_ALIAS_NAME).build(); + final IndexMetadata currentVersionSystem = buildIndexWithAlias(".system1", SYSTEM_ALIAS_NAME, null, Version.CURRENT, true); + final IndexMetadata oldVersionSystem = buildIndexWithAlias(".oldVersionSystem", SYSTEM_ALIAS_NAME, null, random7xVersion, true); + final IndexMetadata regularIndex = buildIndexWithAlias("regular1", SYSTEM_ALIAS_NAME, false, Version.CURRENT, false); + + IndexAbstraction.Alias mixedVersionSystemAndRegular = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + mixedVersionSystemAndRegular.addIndex(oldVersionSystem); + mixedVersionSystemAndRegular.addIndex(regularIndex); + IllegalStateException exception = expectThrows(IllegalStateException.class, + () -> mixedVersionSystemAndRegular.computeAndValidateAliasProperties()); + assertThat(exception.getMessage(), containsString("alias [" + SYSTEM_ALIAS_NAME + + "] refers to both system indices [" + currentVersionSystem.getIndex().getName() + "] and non-system indices: [" + + regularIndex.getIndex().getName() + "], but aliases must refer to either system or non-system indices, not both")); + } - final IndexMetadata regularIndex = buildIndexWithAlias("regular1", systemAlias, false, Version.CURRENT, false); + public void testSystemAliasValidationNewSystemAndRegularFails() { + final AliasMetadata aliasMetadata = new AliasMetadata.Builder(SYSTEM_ALIAS_NAME).build(); + final IndexMetadata currentVersionSystem = buildIndexWithAlias(".system1", SYSTEM_ALIAS_NAME, null, Version.CURRENT, true); + final IndexMetadata regularIndex = buildIndexWithAlias("regular1", SYSTEM_ALIAS_NAME, false, Version.CURRENT, false); + + IndexAbstraction.Alias systemAndRegular = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + systemAndRegular.addIndex(regularIndex); + IllegalStateException exception = expectThrows(IllegalStateException.class, + () -> systemAndRegular.computeAndValidateAliasProperties()); + assertThat(exception.getMessage(), containsString("alias [" + SYSTEM_ALIAS_NAME + + "] refers to both system indices [" + currentVersionSystem.getIndex().getName() + "] and non-system indices: [" + + regularIndex.getIndex().getName() + "], but aliases must refer to either system or non-system indices, not both")); + } - // All system on 8.0+ - { - IndexAbstraction.Alias allSystemCurrent = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); - allSystemCurrent.addIndex(currentVersionSystem2); - allSystemCurrent.computeAndValidateAliasProperties(); // Should be ok - } + public void testSystemAliasOldSystemAndNewRegular() { + final Version random7xVersion = VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, + VersionUtils.getPreviousVersion(Version.V_8_0_0)); + final AliasMetadata aliasMetadata = new AliasMetadata.Builder(SYSTEM_ALIAS_NAME).build(); + final IndexMetadata oldVersionSystem = buildIndexWithAlias(".oldVersionSystem", SYSTEM_ALIAS_NAME, null, random7xVersion, true); + final IndexMetadata regularIndex = buildIndexWithAlias("regular1", SYSTEM_ALIAS_NAME, false, Version.CURRENT, false); - // All system, some from an older version - { - IndexAbstraction.Alias allSystemMixed = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); - allSystemMixed.addIndex(currentVersionSystem2); - allSystemMixed.addIndex(oldVersionSystem); - allSystemMixed.computeAndValidateAliasProperties(); // Should be ok - } + IndexAbstraction.Alias oldAndRegular = new IndexAbstraction.Alias(aliasMetadata, oldVersionSystem); + oldAndRegular.addIndex(regularIndex); + oldAndRegular.computeAndValidateAliasProperties(); // Should be ok + } - // All regular - { - IndexAbstraction.Alias allRegular = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); - allRegular.addIndex(currentVersionSystem2); - allRegular.addIndex(oldVersionSystem); - allRegular.computeAndValidateAliasProperties(); // Should be ok - } + public void testSystemIndexValidationAllRegular() { + final Version random7xVersion = VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, + VersionUtils.getPreviousVersion(Version.V_8_0_0)); + final AliasMetadata aliasMetadata = new AliasMetadata.Builder(SYSTEM_ALIAS_NAME).build(); + final IndexMetadata currentVersionSystem = buildIndexWithAlias(".system1", SYSTEM_ALIAS_NAME, null, Version.CURRENT, true); + final IndexMetadata currentVersionSystem2 = buildIndexWithAlias(".system2", SYSTEM_ALIAS_NAME, null, Version.CURRENT, true); + final IndexMetadata oldVersionSystem = buildIndexWithAlias(".oldVersionSystem", SYSTEM_ALIAS_NAME, null, random7xVersion, true); + + IndexAbstraction.Alias allRegular = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + allRegular.addIndex(currentVersionSystem2); + allRegular.addIndex(oldVersionSystem); + allRegular.computeAndValidateAliasProperties(); // Should be ok + } - // System index from pre-8.0 + regular index should be fine - { - IndexAbstraction.Alias oldAndRegular = new IndexAbstraction.Alias(aliasMetadata, oldVersionSystem); - oldAndRegular.addIndex(regularIndex); - oldAndRegular.computeAndValidateAliasProperties(); // Should be ok - } + public void testSystemAliasValidationAllSystemSomeOld() { + final Version random7xVersion = VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, + VersionUtils.getPreviousVersion(Version.V_8_0_0)); + final AliasMetadata aliasMetadata = new AliasMetadata.Builder(SYSTEM_ALIAS_NAME).build(); + final IndexMetadata currentVersionSystem = buildIndexWithAlias(".system1", SYSTEM_ALIAS_NAME, null, Version.CURRENT, true); + final IndexMetadata currentVersionSystem2 = buildIndexWithAlias(".system2", SYSTEM_ALIAS_NAME, null, Version.CURRENT, true); + final IndexMetadata oldVersionSystem = buildIndexWithAlias(".oldVersionSystem", SYSTEM_ALIAS_NAME, null, random7xVersion, true); + + IndexAbstraction.Alias allSystemMixed = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + allSystemMixed.addIndex(currentVersionSystem2); + allSystemMixed.addIndex(oldVersionSystem); + allSystemMixed.computeAndValidateAliasProperties(); // Should be ok + } - // Current version system index + regular index should fail - { - IndexAbstraction.Alias systemAndRegular = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); - systemAndRegular.addIndex(regularIndex); - IllegalStateException exception = expectThrows(IllegalStateException.class, - () -> systemAndRegular.computeAndValidateAliasProperties()); - assertThat(exception.getMessage(), containsString("alias [" + systemAlias + - "] refers to both system indices [" + currentVersionSystem.getIndex().getName() + "] and non-system indices: [" + - regularIndex.getIndex().getName() + "], but aliases must refer to either system or non-system indices, not both")); - } + public void testSystemAliasValidationAll8x() { + final AliasMetadata aliasMetadata = new AliasMetadata.Builder(SYSTEM_ALIAS_NAME).build(); + final IndexMetadata currentVersionSystem = buildIndexWithAlias(".system1", SYSTEM_ALIAS_NAME, null, Version.CURRENT, true); + final IndexMetadata currentVersionSystem2 = buildIndexWithAlias(".system2", SYSTEM_ALIAS_NAME, null, Version.CURRENT, true); - // Mixed version system indices + regular index should fail - { - IndexAbstraction.Alias mixedVersionSystemAndRegular = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); - mixedVersionSystemAndRegular.addIndex(oldVersionSystem); - mixedVersionSystemAndRegular.addIndex(regularIndex); - IllegalStateException exception = expectThrows(IllegalStateException.class, - () -> mixedVersionSystemAndRegular.computeAndValidateAliasProperties()); - assertThat(exception.getMessage(), containsString("alias [" + systemAlias + - "] refers to both system indices [" + currentVersionSystem.getIndex().getName() + "] and non-system indices: [" + - regularIndex.getIndex().getName() + "], but aliases must refer to either system or non-system indices, not both")); - } + IndexAbstraction.Alias allSystemCurrent = new IndexAbstraction.Alias(aliasMetadata, currentVersionSystem); + allSystemCurrent.addIndex(currentVersionSystem2); + allSystemCurrent.computeAndValidateAliasProperties(); // Should be ok } private IndexMetadata buildIndexWithAlias(String indexName, String aliasName, @Nullable Boolean aliasIsHidden, From 0cf441196d1f21ba4d08bb727f4a346d26a23b53 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 15 Sep 2020 15:13:04 -0600 Subject: [PATCH 100/161] Add origin for Logstash management APIs --- .../org/elasticsearch/xpack/core/ClientHelper.java | 1 + .../logstash/action/TransportDeletePipelineAction.java | 5 ++++- .../logstash/action/TransportGetPipelineAction.java | 5 ++++- .../logstash/action/TransportPutPipelineAction.java | 5 ++++- .../xpack/security/authz/AuthorizationUtils.java | 10 ++++++---- 5 files changed, 19 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java index d5c817f33e95b..76f5956b17a5a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java @@ -70,6 +70,7 @@ public static Map filterSecurityHeaders(Map head public static final String IDP_ORIGIN = "idp"; public static final String STACK_ORIGIN = "stack"; public static final String SEARCHABLE_SNAPSHOTS_ORIGIN = "searchable_snapshots"; + public static final String LOGSTASH_MANAGEMENT_ORIGIN = "logstash_management"; private ClientHelper() {} diff --git a/x-pack/plugin/logstash/src/main/java/org/elasticsearch/xpack/logstash/action/TransportDeletePipelineAction.java b/x-pack/plugin/logstash/src/main/java/org/elasticsearch/xpack/logstash/action/TransportDeletePipelineAction.java index 8286fcd449f27..a4d276111db67 100644 --- a/x-pack/plugin/logstash/src/main/java/org/elasticsearch/xpack/logstash/action/TransportDeletePipelineAction.java +++ b/x-pack/plugin/logstash/src/main/java/org/elasticsearch/xpack/logstash/action/TransportDeletePipelineAction.java @@ -11,11 +11,14 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.Client; +import org.elasticsearch.client.OriginSettingClient; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.tasks.Task; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.logstash.Logstash; +import static org.elasticsearch.xpack.core.ClientHelper.LOGSTASH_MANAGEMENT_ORIGIN; + public class TransportDeletePipelineAction extends HandledTransportAction { private final Client client; @@ -23,7 +26,7 @@ public class TransportDeletePipelineAction extends HandledTransportAction { private static final Logger logger = LogManager.getLogger(TransportGetPipelineAction.class); @@ -43,7 +46,7 @@ public class TransportGetPipelineAction extends HandledTransportAction { private final Client client; @@ -22,7 +25,7 @@ public class TransportPutPipelineAction extends HandledTransportAction Date: Tue, 15 Sep 2020 15:13:46 -0600 Subject: [PATCH 101/161] Always allow system indiex warnings in ML index template tests --- .../MlConfigIndexMappingsFullClusterRestartIT.java | 2 +- .../elasticsearch/upgrades/MlMappingsUpgradeIT.java | 2 +- .../xpack/test/rest/IndexMappingTemplateAsserter.java | 10 ++++------ 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java index 2796a1a9e49e4..c3c450d4f805b 100644 --- a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java +++ b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/MlConfigIndexMappingsFullClusterRestartIT.java @@ -65,7 +65,7 @@ public void testMlConfigIndexMappingsAfterMigration() throws Exception { createAnomalyDetectorJob(NEW_CLUSTER_JOB_ID); // assert that the mappings are updated - IndexMappingTemplateAsserter.assertMlMappingsMatchTemplates(client(), true); + IndexMappingTemplateAsserter.assertMlMappingsMatchTemplates(client()); } } diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java index af1aab75945bd..b0fafc2d40aef 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java @@ -55,7 +55,7 @@ public void testMappingsUpgrade() throws Exception { assertUpgradedAnnotationsMappings(); closeAndReopenTestJob(); assertUpgradedConfigMappings(); - IndexMappingTemplateAsserter.assertMlMappingsMatchTemplates(client(), false); + IndexMappingTemplateAsserter.assertMlMappingsMatchTemplates(client()); break; default: throw new UnsupportedOperationException("Unknown cluster type [" + CLUSTER_TYPE + "]"); diff --git a/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java index d17bc05a154e6..837854c3033bc 100644 --- a/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java +++ b/x-pack/qa/src/main/java/org/elasticsearch/xpack/test/rest/IndexMappingTemplateAsserter.java @@ -51,10 +51,9 @@ public class IndexMappingTemplateAsserter { * effect of a different test running in the cluster. * * @param client The rest client - * @param allowSystemIndexWarnings Whether deprecation warnings for system index access should be allowed/expected. * @throws IOException On error */ - public static void assertMlMappingsMatchTemplates(RestClient client, boolean allowSystemIndexWarnings) throws IOException { + public static void assertMlMappingsMatchTemplates(RestClient client) throws IOException { // Keys that have been dynamically mapped in the .ml-config index // but are not in the template. These can only be fixed with // re-index and should be addressed at the next major upgrade. @@ -83,15 +82,14 @@ public static void assertMlMappingsMatchTemplates(RestClient client, boolean all statsIndexException.add("properties.hyperparameters.properties.regularization_soft_tree_depth_tolerance.type"); statsIndexException.add("properties.hyperparameters.properties.regularization_tree_size_penalty_multiplier.type"); - assertLegacyTemplateMatchesIndexMappings(client, ".ml-config", ".ml-config", false, configIndexExceptions, - allowSystemIndexWarnings); + assertLegacyTemplateMatchesIndexMappings(client, ".ml-config", ".ml-config", false, configIndexExceptions, true); // the true parameter means the index may not have been created - assertLegacyTemplateMatchesIndexMappings(client, ".ml-meta", ".ml-meta", true, Collections.emptySet(), allowSystemIndexWarnings); + assertLegacyTemplateMatchesIndexMappings(client, ".ml-meta", ".ml-meta", true, Collections.emptySet(), true); assertLegacyTemplateMatchesIndexMappings(client, ".ml-stats", ".ml-stats-000001", true, statsIndexException, false); assertLegacyTemplateMatchesIndexMappings(client, ".ml-state", ".ml-state-000001", true, Collections.emptySet(), false); assertLegacyTemplateMatchesIndexMappings(client, ".ml-notifications-000001", ".ml-notifications-000001"); assertLegacyTemplateMatchesIndexMappings(client, ".ml-inference-000003", ".ml-inference-000003", true, Collections.emptySet(), - allowSystemIndexWarnings); + true); // .ml-annotations-6 does not use a template // .ml-anomalies-shared uses a template but will have dynamically updated mappings as new jobs are opened } From 240a6f093b140044ab3ac7f03091646c400a3e9d Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 16 Sep 2020 16:23:04 -0600 Subject: [PATCH 102/161] Use test assumptions instead of setting system properties per review --- qa/smoke-test-http/build.gradle | 5 ----- .../java/org/elasticsearch/http/SystemIndexRestIT.java | 4 ++++ server/build.gradle | 7 ------- .../metadata/IndexNameExpressionResolverTests.java | 10 ++++++++++ 4 files changed, 14 insertions(+), 12 deletions(-) diff --git a/qa/smoke-test-http/build.gradle b/qa/smoke-test-http/build.gradle index d14c92dfc595c..6f3fb39b70c1b 100644 --- a/qa/smoke-test-http/build.gradle +++ b/qa/smoke-test-http/build.gradle @@ -34,9 +34,4 @@ integTest { * other if we allow them to set the number of available processors as it's set-once in Netty. */ systemProperty 'es.set.netty.runtime.available.processors', 'false' - - /* We need to set this here to prevent release builds from failing while the default is different between snapshot and release builds. - * Specifically, this is needed for SystemIndexRestIT, which assumes the value is `deprecated`. - */ - systemProperty 'es.system_index_access_behavior', 'deprecated' } diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java index 99db3b1a597b4..33056d6f4a8d7 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.indices.SystemIndexDescriptor; +import org.elasticsearch.indices.SystemIndices; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.SystemIndexPlugin; import org.elasticsearch.rest.BaseRestHandler; @@ -48,6 +49,7 @@ import java.util.Map; import java.util.function.Supplier; +import static org.elasticsearch.indices.SystemIndices.AccessBehavior.DEPRECATED; import static org.elasticsearch.test.rest.ESRestTestCase.entityAsMap; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasKey; @@ -63,6 +65,8 @@ protected Collection> nodePlugins() { } public void testSystemIndexAccessBlockedByDefault() throws Exception { + assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", + SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); // create index { Request putDocRequest = new Request("POST", "/_sys_index_test/add_doc/42"); diff --git a/server/build.gradle b/server/build.gradle index 1ae0570679d54..fb28a10e0dc84 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -298,10 +298,3 @@ tasks.named("licenseHeaders").configure { excludes << 'org/apache/lucene/search/RegExp87*' excludes << 'org/apache/lucene/search/RegexpQuery87*' } - -tasks.named("test").configure { - /* We need to set this to prevent release builds from failing while the default is different between snapshot and release builds. - * In particular, this is needed for IndexNameExpressionResolverTests, which assumes the value is `deprecated`. - */ - systemProperty 'es.system_index_access_behavior', 'deprecated' -} 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 ade851cf15f9d..b3120fa1e9b7c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -40,6 +40,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.indices.IndexClosedException; import org.elasticsearch.indices.InvalidIndexNameException; +import org.elasticsearch.indices.SystemIndices; import org.elasticsearch.test.ESTestCase; import java.util.Arrays; @@ -57,6 +58,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_HIDDEN_SETTING; import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY; import static org.elasticsearch.common.util.set.Sets.newHashSet; +import static org.elasticsearch.indices.SystemIndices.AccessBehavior.DEPRECATED; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; import static org.hamcrest.Matchers.arrayWithSize; @@ -1861,6 +1863,8 @@ public void testSingleSystemIndexResolutionAllowed() { } public void testFullWildcardSystemIndexResolutionDeprecated() { + assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", + SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(randomFrom("*", "_all")); @@ -1873,6 +1877,8 @@ public void testFullWildcardSystemIndexResolutionDeprecated() { } public void testSingleSystemIndexResolutionDeprecated() { + assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", + SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".ml-meta"); @@ -1885,6 +1891,8 @@ public void testSingleSystemIndexResolutionDeprecated() { } public void testWildcardSystemIndexReslutionSingleMatchDeprecated() { + assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", + SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".w*"); @@ -1897,6 +1905,8 @@ public void testWildcardSystemIndexReslutionSingleMatchDeprecated() { } public void testWildcardSystemIndexResolutionMultipleMatchesDeprecated() { + assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", + SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".ml-*"); From 8fd1e262c1273c8365187a25de8f531d643466a7 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 18 Sep 2020 12:09:20 -0600 Subject: [PATCH 103/161] Do not emit deprecation warnings on total wildcard queries --- .../elasticsearch/http/SystemIndexRestIT.java | 12 +++++- .../metadata/IndexNameExpressionResolver.java | 43 +++++++++++-------- .../IndexNameExpressionResolverTests.java | 5 +-- 3 files changed, 36 insertions(+), 24 deletions(-) diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java index 33056d6f4a8d7..48920b3fdf3d2 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -97,8 +97,16 @@ public void testSystemIndexAccessBlockedByDefault() throws Exception { // And with a partial wildcard assertDeprecationWarningOnAccess(".test-*", SystemIndexTestPlugin.SYSTEM_INDEX_NAME); - // And with a total wildcard - assertDeprecationWarningOnAccess(randomFrom("*", "_all"), SystemIndexTestPlugin.SYSTEM_INDEX_NAME); + // Check that a total wildcard does not trigger a warning + { + String queryPattern = randomFrom("*", "_all"); + Request searchRequest = new Request("GET", "/" + queryPattern + randomFrom("/_count", "/_search")); + searchRequest.setJsonEntity("{\"query\": {\"match\": {\"some_field\": \"some_value\"}}}"); + // Disallow no indices to cause an exception if this resolves to zero indices, so that we're sure it resolved the index + searchRequest.addParameter("allow_no_indices", "false"); + Response response = getRestClient().performRequest(searchRequest); + assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); + } // Try to index a doc directly { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index aeb8a9722e4ba..89d91d0707c1e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -88,7 +88,7 @@ public IndexNameExpressionResolver(ThreadContext threadContext) { */ public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), - isSystemIndexAccessAllowed()); + isSystemIndexAccessAllowedForPatterns(Arrays.asList(request.indices()))); return concreteIndexNames(context, request.indices()); } @@ -106,15 +106,7 @@ public String[] concreteIndexNamesWithSystemIndexAccess(ClusterState state, Indi */ public Index[] concreteIndices(ClusterState state, IndicesRequest request) { Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), - isSystemIndexAccessAllowed()); - return concreteIndices(context, request.indices()); - } - - /** - * Same as {@link #concreteIndices(ClusterState, IndicesRequest)}, but access to system indices is always allowed. - */ - public Index[] concreteIndicesWithSystemIndexAccess(ClusterState state, IndicesRequest request) { - Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), true); + isSystemIndexAccessAllowedForPatterns(Arrays.asList(request.indices()))); return concreteIndices(context, request.indices()); } @@ -132,22 +124,25 @@ public Index[] concreteIndicesWithSystemIndexAccess(ClusterState state, IndicesR * indices options in the context don't allow such a case. */ public String[] concreteIndexNames(ClusterState state, IndicesOptions options, String... indexExpressions) { - Context context = new Context(state, options, isSystemIndexAccessAllowed()); + Context context = new Context(state, options, isSystemIndexAccessAllowedForPatterns(Arrays.asList(indexExpressions))); return concreteIndexNames(context, indexExpressions); } public String[] concreteIndexNames(ClusterState state, IndicesOptions options, boolean includeDataStreams, String... indexExpressions) { - Context context = new Context(state, options, false, false, includeDataStreams, isSystemIndexAccessAllowed()); + Context context = new Context(state, options, false, false, includeDataStreams, + isSystemIndexAccessAllowedForPatterns(Arrays.asList(indexExpressions))); return concreteIndexNames(context, indexExpressions); } public String[] concreteIndexNames(ClusterState state, IndicesOptions options, IndicesRequest request) { - Context context = new Context(state, options, false, false, request.includeDataStreams(), isSystemIndexAccessAllowed()); + Context context = new Context(state, options, false, false, request.includeDataStreams(), + isSystemIndexAccessAllowedForPatterns(Arrays.asList(request.indices()))); return concreteIndexNames(context, request.indices()); } public List dataStreamNames(ClusterState state, IndicesOptions options, String... indexExpressions) { - Context context = new Context(state, options, false, false, true, true, isSystemIndexAccessAllowed()); + Context context = new Context(state, options, false, false, true, true, + isSystemIndexAccessAllowedForPatterns(Arrays.asList(indexExpressions))); if (indexExpressions == null || indexExpressions.length == 0) { indexExpressions = new String[]{"*"}; } @@ -179,7 +174,8 @@ public Index[] concreteIndices(ClusterState state, IndicesOptions options, Strin } public Index[] concreteIndices(ClusterState state, IndicesOptions options, boolean includeDataStreams, String... indexExpressions) { - Context context = new Context(state, options, false, false, includeDataStreams, isSystemIndexAccessAllowed()); + Context context = new Context(state, options, false, false, includeDataStreams, + isSystemIndexAccessAllowedForPatterns(Arrays.asList(indexExpressions))); return concreteIndices(context, indexExpressions); } @@ -197,7 +193,7 @@ public Index[] concreteIndices(ClusterState state, IndicesOptions options, boole */ public Index[] concreteIndices(ClusterState state, IndicesRequest request, long startTime) { Context context = new Context(state, request.indicesOptions(), startTime, false, false, request.includeDataStreams(), false, - isSystemIndexAccessAllowed()); + isSystemIndexAccessAllowedForPatterns(Arrays.asList(request.indices()))); return concreteIndices(context, request.indices()); } @@ -473,7 +469,8 @@ public String resolveDateMathExpression(String dateExpression) { * Resolve an array of expressions to the set of indices and aliases that these expressions match. */ public Set resolveExpressions(ClusterState state, String... expressions) { - Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true, false, true, isSystemIndexAccessAllowed()); + Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true, false, true, + isSystemIndexAccessAllowedForPatterns(Arrays.asList(expressions))); List resolvedExpressions = Arrays.asList(expressions); for (ExpressionResolver expressionResolver : expressionResolvers) { resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions); @@ -567,7 +564,8 @@ public String[] indexAliases(ClusterState state, String index, Predicate> resolveSearchRouting(ClusterState state, @Nullable String routing, String... expressions) { List resolvedExpressions = expressions != null ? Arrays.asList(expressions) : Collections.emptyList(); - Context context = new Context(state, IndicesOptions.lenientExpandOpen(), false, false, true, isSystemIndexAccessAllowed()); + Context context = new Context(state, IndicesOptions.lenientExpandOpen(), false, false, true, + isSystemIndexAccessAllowedForPatterns(Arrays.asList(expressions))); for (ExpressionResolver expressionResolver : expressionResolvers) { resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions); } @@ -723,6 +721,15 @@ private boolean isSystemIndexAccessAllowed() { return Booleans.parseBoolean(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY), true); } + private boolean isSystemIndexAccessAllowedForPatterns(List patterns) { + if (patterns == null || 1 != patterns.size()) { + return isSystemIndexAccessAllowed(); + } else { + final String pattern = patterns.get(0); + return isSystemIndexAccessAllowed() || pattern == null || Metadata.ALL.equals(pattern) || Regex.isMatchAllPattern(pattern); + } + } + public static class Context { private final ClusterState state; 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 b3120fa1e9b7c..6423b4eda312c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -1867,13 +1867,10 @@ public void testFullWildcardSystemIndexResolutionDeprecated() { SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); - SearchRequest request = new SearchRequest(randomFrom("*", "_all")); + SearchRequest request = new SearchRequest(randomFrom("*")); List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); - assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff, .watches], but in a future major version, " + - "direct access to system indices will be prevented by default"); - } public void testSingleSystemIndexResolutionDeprecated() { From 9459027136e51beb982d12d26a3f54e7a8698517 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 18 Sep 2020 12:09:45 -0600 Subject: [PATCH 104/161] Remove system index access by default from APIs which no longer need it --- .../alias/get/TransportGetAliasesAction.java | 4 +-- .../metadata/IndexNameExpressionResolver.java | 28 ++++++++++--------- .../admin/indices/RestGetAliasesAction.java | 6 ---- .../rest/action/cat/AbstractCatAction.java | 5 ---- .../rest/RestDataStreamsStatsAction.java | 6 ---- 5 files changed, 17 insertions(+), 32 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java index b1305ba2acf52..4fbf8e84e62bd 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java @@ -57,7 +57,7 @@ protected String executor() { @Override protected ClusterBlockException checkBlock(GetAliasesRequest request, ClusterState state) { return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, - indexNameExpressionResolver.concreteIndexNamesWithSystemIndexAccess(state, request)); + indexNameExpressionResolver.concreteIndexNames(state, request)); } @Override @@ -67,7 +67,7 @@ protected GetAliasesResponse read(StreamInput in) throws IOException { @Override protected void masterOperation(Task task, GetAliasesRequest request, ClusterState state, ActionListener listener) { - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNamesWithSystemIndexAccess(state, request); + String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); ImmutableOpenMap> aliases = state.metadata().findAliases(request, concreteIndices); listener.onResponse(new GetAliasesResponse(postProcess(request, concreteIndices, aliases))); } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 89d91d0707c1e..8b555f0092fde 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -88,7 +88,7 @@ public IndexNameExpressionResolver(ThreadContext threadContext) { */ public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), - isSystemIndexAccessAllowedForPatterns(Arrays.asList(request.indices()))); + isSystemIndexAccessAllowedForPatterns(request.indices())); return concreteIndexNames(context, request.indices()); } @@ -106,7 +106,7 @@ public String[] concreteIndexNamesWithSystemIndexAccess(ClusterState state, Indi */ public Index[] concreteIndices(ClusterState state, IndicesRequest request) { Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), - isSystemIndexAccessAllowedForPatterns(Arrays.asList(request.indices()))); + isSystemIndexAccessAllowedForPatterns(request.indices())); return concreteIndices(context, request.indices()); } @@ -124,25 +124,25 @@ public Index[] concreteIndices(ClusterState state, IndicesRequest request) { * indices options in the context don't allow such a case. */ public String[] concreteIndexNames(ClusterState state, IndicesOptions options, String... indexExpressions) { - Context context = new Context(state, options, isSystemIndexAccessAllowedForPatterns(Arrays.asList(indexExpressions))); + Context context = new Context(state, options, isSystemIndexAccessAllowedForPatterns(indexExpressions)); return concreteIndexNames(context, indexExpressions); } public String[] concreteIndexNames(ClusterState state, IndicesOptions options, boolean includeDataStreams, String... indexExpressions) { Context context = new Context(state, options, false, false, includeDataStreams, - isSystemIndexAccessAllowedForPatterns(Arrays.asList(indexExpressions))); + isSystemIndexAccessAllowedForPatterns(indexExpressions)); return concreteIndexNames(context, indexExpressions); } public String[] concreteIndexNames(ClusterState state, IndicesOptions options, IndicesRequest request) { Context context = new Context(state, options, false, false, request.includeDataStreams(), - isSystemIndexAccessAllowedForPatterns(Arrays.asList(request.indices()))); + isSystemIndexAccessAllowedForPatterns(request.indices())); return concreteIndexNames(context, request.indices()); } public List dataStreamNames(ClusterState state, IndicesOptions options, String... indexExpressions) { Context context = new Context(state, options, false, false, true, true, - isSystemIndexAccessAllowedForPatterns(Arrays.asList(indexExpressions))); + isSystemIndexAccessAllowedForPatterns(indexExpressions)); if (indexExpressions == null || indexExpressions.length == 0) { indexExpressions = new String[]{"*"}; } @@ -175,7 +175,7 @@ public Index[] concreteIndices(ClusterState state, IndicesOptions options, Strin public Index[] concreteIndices(ClusterState state, IndicesOptions options, boolean includeDataStreams, String... indexExpressions) { Context context = new Context(state, options, false, false, includeDataStreams, - isSystemIndexAccessAllowedForPatterns(Arrays.asList(indexExpressions))); + isSystemIndexAccessAllowedForPatterns(indexExpressions)); return concreteIndices(context, indexExpressions); } @@ -193,7 +193,7 @@ public Index[] concreteIndices(ClusterState state, IndicesOptions options, boole */ public Index[] concreteIndices(ClusterState state, IndicesRequest request, long startTime) { Context context = new Context(state, request.indicesOptions(), startTime, false, false, request.includeDataStreams(), false, - isSystemIndexAccessAllowedForPatterns(Arrays.asList(request.indices()))); + isSystemIndexAccessAllowedForPatterns(request.indices())); return concreteIndices(context, request.indices()); } @@ -470,7 +470,7 @@ public String resolveDateMathExpression(String dateExpression) { */ public Set resolveExpressions(ClusterState state, String... expressions) { Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true, false, true, - isSystemIndexAccessAllowedForPatterns(Arrays.asList(expressions))); + isSystemIndexAccessAllowedForPatterns(expressions)); List resolvedExpressions = Arrays.asList(expressions); for (ExpressionResolver expressionResolver : expressionResolvers) { resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions); @@ -565,7 +565,7 @@ public String[] indexAliases(ClusterState state, String index, Predicate> resolveSearchRouting(ClusterState state, @Nullable String routing, String... expressions) { List resolvedExpressions = expressions != null ? Arrays.asList(expressions) : Collections.emptyList(); Context context = new Context(state, IndicesOptions.lenientExpandOpen(), false, false, true, - isSystemIndexAccessAllowedForPatterns(Arrays.asList(expressions))); + isSystemIndexAccessAllowedForPatterns(expressions)); for (ExpressionResolver expressionResolver : expressionResolvers) { resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions); } @@ -721,11 +721,13 @@ private boolean isSystemIndexAccessAllowed() { return Booleans.parseBoolean(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY), true); } - private boolean isSystemIndexAccessAllowedForPatterns(List patterns) { - if (patterns == null || 1 != patterns.size()) { + private boolean isSystemIndexAccessAllowedForPatterns(String[] patterns) { + if (patterns == null) { + return true; + } else if (1 != patterns.length) { return isSystemIndexAccessAllowed(); } else { - final String pattern = patterns.get(0); + final String pattern = patterns[0]; return isSystemIndexAccessAllowed() || pattern == null || Metadata.ALL.equals(pattern) || Regex.isMatchAllPattern(pattern); } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetAliasesAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetAliasesAction.java index a28697a148a21..2e5bc82d9b05e 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetAliasesAction.java @@ -72,12 +72,6 @@ public String getName() { return "get_aliases_action"; } - @Override - public boolean allowSystemIndexAccessByDefault() { - // If not given an index (only an alias), resolves `_all`, which causes deprecation warnings. - return true; - } - static RestResponse buildRestResponse(boolean aliasesExplicitlyRequested, String[] requestedAliases, ImmutableOpenMap> responseAliasMap, XContentBuilder builder) throws Exception { diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java index fedbf038b5562..a3bf7ff3fbc98 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/AbstractCatAction.java @@ -42,11 +42,6 @@ public abstract class AbstractCatAction extends BaseRestHandler { protected abstract Table getTableWithHeader(RestRequest request); - @Override - public boolean allowSystemIndexAccessByDefault() { - return true; - } - @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { boolean helpWanted = request.paramAsBoolean("help", false); diff --git a/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/rest/RestDataStreamsStatsAction.java b/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/rest/RestDataStreamsStatsAction.java index 13b676615064d..f6f93fd05fbdb 100644 --- a/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/rest/RestDataStreamsStatsAction.java +++ b/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/rest/RestDataStreamsStatsAction.java @@ -29,12 +29,6 @@ public List routes() { ); } - @Override - public boolean allowSystemIndexAccessByDefault() { - // Does not actually access system indices, but does resolve `_all`, which can cause confusing deprecation warnings. - return true; - } - @Override protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { DataStreamsStatsAction.Request dataStreamsStatsRequest = new DataStreamsStatsAction.Request(); From 8e990ec114e24d107a4f4cb9b70282e1b61dd623 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 18 Sep 2020 14:48:38 -0600 Subject: [PATCH 105/161] First cut of security integration for _all handling --- .../metadata/IndexNameExpressionResolver.java | 14 +++++++++++--- .../xpack/security/authz/AuthorizationService.java | 2 +- .../security/authz/IndicesAndAliasesResolver.java | 12 +++++++++++- .../authz/IndicesAndAliasesResolverTests.java | 3 ++- 4 files changed, 25 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 8b555f0092fde..b30e57612ae89 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -70,6 +70,7 @@ public class IndexNameExpressionResolver { public static final String EXCLUDED_DATA_STREAMS_KEY = "es.excluded_ds"; public static final String SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY = "_system_index_access_allowed"; + public static final String ORIGINAL_INDICES_HEADER_KEY = "_original_indices"; public static final Version SYSTEM_INDEX_ENFORCEMENT_VERSION = Version.V_8_0_0; private final DateMathExpressionResolver dateMathExpressionResolver = new DateMathExpressionResolver(); @@ -722,12 +723,19 @@ private boolean isSystemIndexAccessAllowed() { } private boolean isSystemIndexAccessAllowedForPatterns(String[] patterns) { - if (patterns == null) { + String[] checkPatterns = patterns; + + String originalPatterns = threadContext.getHeader(ORIGINAL_INDICES_HEADER_KEY); + if (originalPatterns != null) { + checkPatterns = originalPatterns.split(","); + } + + if (checkPatterns == null) { return true; - } else if (1 != patterns.length) { + } else if (1 != checkPatterns.length) { return isSystemIndexAccessAllowed(); } else { - final String pattern = patterns[0]; + final String pattern = checkPatterns[0]; return isSystemIndexAccessAllowed() || pattern == null || Metadata.ALL.equals(pattern) || Regex.isMatchAllPattern(pattern); } } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java index 6d7d4d896c009..ab077338fc34e 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java @@ -127,7 +127,7 @@ public AuthorizationService(Settings settings, CompositeRolesStore rolesStore, C IndexNameExpressionResolver resolver) { this.clusterService = clusterService; this.auditTrailService = auditTrailService; - this.indicesAndAliasesResolver = new IndicesAndAliasesResolver(settings, clusterService, resolver); + this.indicesAndAliasesResolver = new IndicesAndAliasesResolver(settings, clusterService, resolver, threadPool.getThreadContext()); this.authcFailureHandler = authcFailureHandler; this.threadContext = threadPool.getThreadContext(); this.anonymousUser = anonymousUser; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java index 64c2d33ff098d..4838d88a8358f 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.protocol.xpack.graph.GraphExploreRequest; import org.elasticsearch.transport.RemoteClusterAware; @@ -44,6 +45,7 @@ import java.util.concurrent.CopyOnWriteArraySet; import java.util.stream.Collectors; +import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ORIGINAL_INDICES_HEADER_KEY; import static org.elasticsearch.xpack.core.security.authz.IndicesAndAliasesResolverField.NO_INDEX_PLACEHOLDER; class IndicesAndAliasesResolver { @@ -55,11 +57,14 @@ class IndicesAndAliasesResolver { private final IndexNameExpressionResolver nameExpressionResolver; private final IndexAbstractionResolver indexAbstractionResolver; private final RemoteClusterResolver remoteClusterResolver; + private final ThreadContext threadContext; - IndicesAndAliasesResolver(Settings settings, ClusterService clusterService, IndexNameExpressionResolver resolver) { + IndicesAndAliasesResolver(Settings settings, ClusterService clusterService, IndexNameExpressionResolver resolver, + ThreadContext threadContext) { this.nameExpressionResolver = resolver; this.indexAbstractionResolver = new IndexAbstractionResolver(resolver); this.remoteClusterResolver = new RemoteClusterResolver(settings, clusterService.getClusterSettings()); + this.threadContext = threadContext; } /** @@ -114,6 +119,11 @@ ResolvedIndices resolve(TransportRequest request, Metadata metadata, List authorizedIndices) { final ResolvedIndices.Builder resolvedIndicesBuilder = new ResolvedIndices.Builder(); boolean indicesReplacedWithNoIndices = false; + // We only care about the indices specified in the first layer of the request, as that's what the user actually asked for. + if (threadContext.getHeader(ORIGINAL_INDICES_HEADER_KEY) == null) { + threadContext.putHeader(ORIGINAL_INDICES_HEADER_KEY, + indicesRequest.indices() == null ? Metadata.ALL : String.join(",", indicesRequest.indices())); + } if (indicesRequest instanceof PutMappingRequest && ((PutMappingRequest) indicesRequest).getConcreteIndex() != null) { /* * This is a special case since PutMappingRequests from dynamic mapping updates have a concrete index diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java index 9c3d7b036c3e6..ff4510b272250 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java @@ -267,8 +267,9 @@ public void setup() { ClusterService clusterService = mock(ClusterService.class); when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); + final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); defaultIndicesResolver = - new IndicesAndAliasesResolver(settings, clusterService, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); + new IndicesAndAliasesResolver(settings, clusterService, new IndexNameExpressionResolver(threadContext), threadContext); } public void testDashIndicesAreAllowedInShardLevelRequests() { From 24682c2532af15d75ac917768e8dd033b2f54fa2 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 18 Sep 2020 16:19:00 -0600 Subject: [PATCH 106/161] Add yet another allow_system_index_access flag as necessary --- .../src/test/resources/rest-api-spec/test/ml/inference_crud.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_crud.yml index 31bda1d9a271e..0b614ace6ca3d 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_crud.yml @@ -12,6 +12,7 @@ setup: index: id: trained_model_metadata-a-regression-model-0 index: .ml-inference-000003 + allow_system_index_access: true body: model_id: "a-regression-model-0" doc_type: "trained_model_metadata" From 3218750bd10983cbec045d2f629ff9c341e0b252 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 18 Sep 2020 16:35:20 -0600 Subject: [PATCH 107/161] Add yet more allow_system_index_access flag as necessary --- .../src/test/resources/rest-api-spec/test/ml/index_layout.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml index 2b8a9063b4ad3..f186889466cd5 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml @@ -104,6 +104,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: + allow_system_index_access: true name: ".ml-state-write" - is_true: '' @@ -699,6 +700,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: + allow_system_index_access: true name: ".ml-state-write" - is_true: '' From 50a013c1b4e6e96f78d0b278c8b1e04a492a728c Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 21 Sep 2020 16:14:41 -0600 Subject: [PATCH 108/161] Check if a deprecation warning should be emitted rather than allowing system index access for _all --- .../metadata/IndexNameExpressionResolver.java | 42 ++++++++----------- .../authz/IndicesAndAliasesResolver.java | 4 +- 2 files changed, 20 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index b30e57612ae89..eec97ae7a038a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -88,8 +88,7 @@ public IndexNameExpressionResolver(ThreadContext threadContext) { * are encapsulated in the specified request. */ public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { - Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), - isSystemIndexAccessAllowedForPatterns(request.indices())); + Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), isSystemIndexAccessAllowed()); return concreteIndexNames(context, request.indices()); } @@ -106,8 +105,7 @@ public String[] concreteIndexNamesWithSystemIndexAccess(ClusterState state, Indi * are encapsulated in the specified request and resolves data streams. */ public Index[] concreteIndices(ClusterState state, IndicesRequest request) { - Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), - isSystemIndexAccessAllowedForPatterns(request.indices())); + Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), isSystemIndexAccessAllowed()); return concreteIndices(context, request.indices()); } @@ -125,25 +123,22 @@ public Index[] concreteIndices(ClusterState state, IndicesRequest request) { * indices options in the context don't allow such a case. */ public String[] concreteIndexNames(ClusterState state, IndicesOptions options, String... indexExpressions) { - Context context = new Context(state, options, isSystemIndexAccessAllowedForPatterns(indexExpressions)); + Context context = new Context(state, options, isSystemIndexAccessAllowed()); return concreteIndexNames(context, indexExpressions); } public String[] concreteIndexNames(ClusterState state, IndicesOptions options, boolean includeDataStreams, String... indexExpressions) { - Context context = new Context(state, options, false, false, includeDataStreams, - isSystemIndexAccessAllowedForPatterns(indexExpressions)); + Context context = new Context(state, options, false, false, includeDataStreams, isSystemIndexAccessAllowed()); return concreteIndexNames(context, indexExpressions); } public String[] concreteIndexNames(ClusterState state, IndicesOptions options, IndicesRequest request) { - Context context = new Context(state, options, false, false, request.includeDataStreams(), - isSystemIndexAccessAllowedForPatterns(request.indices())); + Context context = new Context(state, options, false, false, request.includeDataStreams(), isSystemIndexAccessAllowed()); return concreteIndexNames(context, request.indices()); } public List dataStreamNames(ClusterState state, IndicesOptions options, String... indexExpressions) { - Context context = new Context(state, options, false, false, true, true, - isSystemIndexAccessAllowedForPatterns(indexExpressions)); + Context context = new Context(state, options, false, false, true, true, isSystemIndexAccessAllowed()); if (indexExpressions == null || indexExpressions.length == 0) { indexExpressions = new String[]{"*"}; } @@ -176,7 +171,7 @@ public Index[] concreteIndices(ClusterState state, IndicesOptions options, Strin public Index[] concreteIndices(ClusterState state, IndicesOptions options, boolean includeDataStreams, String... indexExpressions) { Context context = new Context(state, options, false, false, includeDataStreams, - isSystemIndexAccessAllowedForPatterns(indexExpressions)); + isSystemIndexAccessAllowed()); return concreteIndices(context, indexExpressions); } @@ -193,8 +188,7 @@ public Index[] concreteIndices(ClusterState state, IndicesOptions options, boole * indices options in the context don't allow such a case. */ public Index[] concreteIndices(ClusterState state, IndicesRequest request, long startTime) { - Context context = new Context(state, request.indicesOptions(), startTime, false, false, request.includeDataStreams(), false, - isSystemIndexAccessAllowedForPatterns(request.indices())); + Context context = new Context(state, request.indicesOptions(), startTime, false, false, request.includeDataStreams(), false, isSystemIndexAccessAllowed()); return concreteIndices(context, request.indices()); } @@ -314,12 +308,12 @@ Index[] concreteIndices(Context context, String... indexExpressions) { } throw infe; } - checkSystemIndexAccess(context, metadata, concreteIndices); + checkSystemIndexAccess(context, metadata, concreteIndices, indexExpressions); return concreteIndices.toArray(new Index[concreteIndices.size()]); } - private static void checkSystemIndexAccess(Context context, Metadata metadata, Set concreteIndices) { - if (context.isSystemIndexAccessAllowed() == false) { + private void checkSystemIndexAccess(Context context, Metadata metadata, Set concreteIndices, String[] originalPatterns) { + if (context.isSystemIndexAccessAllowed() == false && shouldEmitDeprecationWarning(originalPatterns)) { final List resolvedSystemIndices = concreteIndices.stream() .map(metadata::index) .filter(IndexMetadata::isSystem) @@ -470,8 +464,7 @@ public String resolveDateMathExpression(String dateExpression) { * Resolve an array of expressions to the set of indices and aliases that these expressions match. */ public Set resolveExpressions(ClusterState state, String... expressions) { - Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true, false, true, - isSystemIndexAccessAllowedForPatterns(expressions)); + Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true, false, true, isSystemIndexAccessAllowed()); List resolvedExpressions = Arrays.asList(expressions); for (ExpressionResolver expressionResolver : expressionResolvers) { resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions); @@ -565,8 +558,7 @@ public String[] indexAliases(ClusterState state, String index, Predicate> resolveSearchRouting(ClusterState state, @Nullable String routing, String... expressions) { List resolvedExpressions = expressions != null ? Arrays.asList(expressions) : Collections.emptyList(); - Context context = new Context(state, IndicesOptions.lenientExpandOpen(), false, false, true, - isSystemIndexAccessAllowedForPatterns(expressions)); + Context context = new Context(state, IndicesOptions.lenientExpandOpen(), false, false, true, isSystemIndexAccessAllowed()); for (ExpressionResolver expressionResolver : expressionResolvers) { resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions); } @@ -722,7 +714,7 @@ private boolean isSystemIndexAccessAllowed() { return Booleans.parseBoolean(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY), true); } - private boolean isSystemIndexAccessAllowedForPatterns(String[] patterns) { + private boolean shouldEmitDeprecationWarning(String[] patterns) { String[] checkPatterns = patterns; String originalPatterns = threadContext.getHeader(ORIGINAL_INDICES_HEADER_KEY); @@ -731,12 +723,12 @@ private boolean isSystemIndexAccessAllowedForPatterns(String[] patterns) { } if (checkPatterns == null) { - return true; + return false; } else if (1 != checkPatterns.length) { - return isSystemIndexAccessAllowed(); + return isSystemIndexAccessAllowed() == false; } else { final String pattern = checkPatterns[0]; - return isSystemIndexAccessAllowed() || pattern == null || Metadata.ALL.equals(pattern) || Regex.isMatchAllPattern(pattern); + return (isSystemIndexAccessAllowed() || pattern == null || Metadata.ALL.equals(pattern) || Regex.isMatchAllPattern(pattern)) == false; } } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java index 4838d88a8358f..f4b34915f0cba 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java @@ -122,7 +122,9 @@ ResolvedIndices resolveIndicesAndAliases(IndicesRequest indicesRequest, Metadata // We only care about the indices specified in the first layer of the request, as that's what the user actually asked for. if (threadContext.getHeader(ORIGINAL_INDICES_HEADER_KEY) == null) { threadContext.putHeader(ORIGINAL_INDICES_HEADER_KEY, - indicesRequest.indices() == null ? Metadata.ALL : String.join(",", indicesRequest.indices())); + indicesRequest.indices() == null || indicesRequest.indices().length == 0 + ? Metadata.ALL + : String.join(",", indicesRequest.indices())); } if (indicesRequest instanceof PutMappingRequest && ((PutMappingRequest) indicesRequest).getConcreteIndex() != null) { /* From c8d55a5e921c7b46aa39a7bed2ca3254333e1a85 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 21 Sep 2020 16:15:06 -0600 Subject: [PATCH 109/161] Remove header drop from IndexResolver --- .../org/elasticsearch/xpack/ql/index/IndexResolver.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java index d1d67bdbc77f4..d75d38529958b 100644 --- a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/index/IndexResolver.java @@ -17,7 +17,6 @@ import org.elasticsearch.action.fieldcaps.FieldCapabilities; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; -import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions.Option; import org.elasticsearch.action.support.IndicesOptions.WildcardStates; @@ -231,12 +230,9 @@ private void resolveIndices(String[] indices, String javaRegex, GetAliasesRespon indexRequest.indicesOptions(FROZEN_INDICES_OPTIONS); } - // Drop the response headers from this request, because this very easily triggers a spurious deprecation warning due to - // resolving system indices, despite them being filtered out later. client.admin().indices().getIndex(indexRequest, - new ContextPreservingActionListener<>(client.threadPool().getThreadContext().newRestorableContext(false), - wrap(response -> filterResults(javaRegex, aliases, response, retrieveIndices, retrieveFrozenIndices, listener), - listener::onFailure))); + wrap(response -> filterResults(javaRegex, aliases, response, retrieveIndices, retrieveFrozenIndices, listener), + listener::onFailure)); } else { filterResults(javaRegex, aliases, null, false, false, listener); From 9c15ae8861ded7eb04fdf45f2d78cc9785890c86 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 21 Sep 2020 17:16:22 -0600 Subject: [PATCH 110/161] Set original indices header OUTSIDE of the context preserving listener --- .../xpack/security/authz/AuthorizationService.java | 14 ++++++++++++++ .../security/authz/IndicesAndAliasesResolver.java | 8 -------- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java index ab077338fc34e..5b041a8e61f52 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java @@ -11,6 +11,7 @@ import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.StepListener; import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesAction; @@ -87,6 +88,7 @@ import java.util.function.Consumer; import static org.elasticsearch.action.support.ContextPreservingActionListener.wrapPreservingContext; +import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ORIGINAL_INDICES_HEADER_KEY; import static org.elasticsearch.common.Strings.collectionToCommaDelimitedString; import static org.elasticsearch.xpack.core.security.SecurityField.setting; import static org.elasticsearch.xpack.core.security.authz.AuthorizationServiceField.ACTION_SCOPE_AUTHORIZATION_KEYS; @@ -281,6 +283,7 @@ private void authorizeAction(final RequestInfo requestInfo, final String request } })); }); + storeOriginalIndicesHeader(requestInfo); authzEngine.authorizeIndexAction(requestInfo, authzInfo, resolvedIndicesAsyncSupplier, metadata.getIndicesLookup(), wrapPreservingContext(new AuthorizationResultListener<>(result -> handleIndexActionAuthorizationResult(result, requestInfo, requestId, authzInfo, authzEngine, authorizedIndicesSupplier, @@ -293,6 +296,17 @@ private void authorizeAction(final RequestInfo requestInfo, final String request } } + private void storeOriginalIndicesHeader(RequestInfo requestInfo) { + // We only care about the indices specified in the first layer of the request, as that's what the user actually asked for. + if (requestInfo.getRequest() instanceof IndicesRequest && threadContext.getHeader(ORIGINAL_INDICES_HEADER_KEY) == null) { + IndicesRequest indicesRequest = (IndicesRequest) requestInfo.getRequest(); + threadContext.putHeader(ORIGINAL_INDICES_HEADER_KEY, + indicesRequest.indices() == null || indicesRequest.indices().length == 0 + ? Metadata.ALL + : String.join(",", indicesRequest.indices())); + } + } + private void handleIndexActionAuthorizationResult(final IndexAuthorizationResult result, final RequestInfo requestInfo, final String requestId, final AuthorizationInfo authzInfo, final AuthorizationEngine authzEngine, diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java index f4b34915f0cba..6a5212ba465b9 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java @@ -45,7 +45,6 @@ import java.util.concurrent.CopyOnWriteArraySet; import java.util.stream.Collectors; -import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ORIGINAL_INDICES_HEADER_KEY; import static org.elasticsearch.xpack.core.security.authz.IndicesAndAliasesResolverField.NO_INDEX_PLACEHOLDER; class IndicesAndAliasesResolver { @@ -119,13 +118,6 @@ ResolvedIndices resolve(TransportRequest request, Metadata metadata, List authorizedIndices) { final ResolvedIndices.Builder resolvedIndicesBuilder = new ResolvedIndices.Builder(); boolean indicesReplacedWithNoIndices = false; - // We only care about the indices specified in the first layer of the request, as that's what the user actually asked for. - if (threadContext.getHeader(ORIGINAL_INDICES_HEADER_KEY) == null) { - threadContext.putHeader(ORIGINAL_INDICES_HEADER_KEY, - indicesRequest.indices() == null || indicesRequest.indices().length == 0 - ? Metadata.ALL - : String.join(",", indicesRequest.indices())); - } if (indicesRequest instanceof PutMappingRequest && ((PutMappingRequest) indicesRequest).getConcreteIndex() != null) { /* * This is a special case since PutMappingRequests from dynamic mapping updates have a concrete index From dfb6c78e44d8f9ffefc676666c4d9c7fcb9134cd Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 21 Sep 2020 17:22:37 -0600 Subject: [PATCH 111/161] Line lengths --- .../metadata/IndexNameExpressionResolver.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index eec97ae7a038a..a6e69cb4b014c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -88,7 +88,8 @@ public IndexNameExpressionResolver(ThreadContext threadContext) { * are encapsulated in the specified request. */ public String[] concreteIndexNames(ClusterState state, IndicesRequest request) { - Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), isSystemIndexAccessAllowed()); + Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), + isSystemIndexAccessAllowed()); return concreteIndexNames(context, request.indices()); } @@ -105,7 +106,8 @@ public String[] concreteIndexNamesWithSystemIndexAccess(ClusterState state, Indi * are encapsulated in the specified request and resolves data streams. */ public Index[] concreteIndices(ClusterState state, IndicesRequest request) { - Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), isSystemIndexAccessAllowed()); + Context context = new Context(state, request.indicesOptions(), false, false, request.includeDataStreams(), + isSystemIndexAccessAllowed()); return concreteIndices(context, request.indices()); } @@ -188,7 +190,8 @@ public Index[] concreteIndices(ClusterState state, IndicesOptions options, boole * indices options in the context don't allow such a case. */ public Index[] concreteIndices(ClusterState state, IndicesRequest request, long startTime) { - Context context = new Context(state, request.indicesOptions(), startTime, false, false, request.includeDataStreams(), false, isSystemIndexAccessAllowed()); + Context context = new Context(state, request.indicesOptions(), startTime, false, false, request.includeDataStreams(), false, + isSystemIndexAccessAllowed()); return concreteIndices(context, request.indices()); } @@ -728,7 +731,8 @@ private boolean shouldEmitDeprecationWarning(String[] patterns) { return isSystemIndexAccessAllowed() == false; } else { final String pattern = checkPatterns[0]; - return (isSystemIndexAccessAllowed() || pattern == null || Metadata.ALL.equals(pattern) || Regex.isMatchAllPattern(pattern)) == false; + final boolean isAllPattern = pattern == null || Metadata.ALL.equals(pattern) || Regex.isMatchAllPattern(pattern); + return (isSystemIndexAccessAllowed() || isAllPattern) == false; } } From 9f7790f36c6688dbde32f6cebd3c27185a5711b1 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 23 Sep 2020 16:52:50 -0600 Subject: [PATCH 112/161] Replace allow_system_index_access flags in Java tests with warnings handlers --- .../upgrades/FullClusterRestartIT.java | 12 +++++----- .../upgrades/SystemIndicesUpgradeIT.java | 11 ++++----- .../elasticsearch/http/SystemIndexRestIT.java | 23 ++++++++++--------- .../test/rest/ESRestTestCase.java | 19 +++++++++++---- .../ml/integration/InferenceIngestIT.java | 19 +++++++++++++-- .../TransformConfigurationIndexIT.java | 3 ++- .../integration/TransformInternalIndexIT.java | 12 ++++++---- .../integration/TransformRestTestCase.java | 3 ++- .../integration/TransformRobustnessIT.java | 4 +++- .../integration/TransformUsageIT.java | 4 +++- .../xpack/watcher/WatcherRestTestCase.java | 7 +++++- .../watcher/WatcherYamlSuiteTestCase.java | 7 +++++- .../MonitoringWithWatcherRestIT.java | 17 ++++++++++++-- .../xpack/restart/FullClusterRestartIT.java | 6 +++-- .../upgrades/MlMappingsUpgradeIT.java | 3 ++- 15 files changed, 105 insertions(+), 45 deletions(-) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index c08acfdc8e97f..74bb27d5a3603 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -61,7 +61,6 @@ import static org.elasticsearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.rest.RestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -1428,11 +1427,12 @@ public void testSystemIndexMetadataIsUpgraded() throws Exception { Request getTasksIndex = new Request("GET", "/.tasks"); getTasksIndex.addParameter("allow_no_indices", "false"); - // We run "upgrade" tests from the current version as well, so if we're on a recent enough version we need to specify that - // we need system index access here. - if (minimumNodeVersion().onOrAfter(ALLOW_SYSTEM_INDEX_ADDED_VERSION)) { - getTasksIndex.addParameter("allow_system_index_access", "true"); - } + getTasksIndex.setOptions(expectVersionSpecificWarnings(v -> { + final String systemIndexWarning = "this request accesses system indices: [.tasks], but in a future major version, direct " + + "access to system indices will be prevented by default"; + v.current(systemIndexWarning); + v.compatible(systemIndexWarning); + })); assertBusy(() -> { try { assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java index 076406cd04584..f9ad83a29d0ec 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java @@ -27,7 +27,6 @@ import java.util.Map; import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ENFORCEMENT_VERSION; -import static org.elasticsearch.rest.RestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -72,11 +71,11 @@ public void testSystemIndicesUpgrades() throws Exception { Request getTasksIndex = new Request("GET", "/.tasks"); getTasksIndex.addParameter("allow_no_indices", "false"); - // We run "upgrade" tests from the current version as well, so if we're on a recent enough version we need to specify that - // we need system index access here. - if (minimumNodeVersion().onOrAfter(ALLOW_SYSTEM_INDEX_ADDED_VERSION)) { - getTasksIndex.addParameter("allow_system_index_access", "true"); - } + getTasksIndex.setOptions(expectVersionSpecificWarnings(v -> { + final String systemIndexWarning = "this request accesses system indices: [.tasks], but in a future major version, direct " + + "access to system indices will be prevented by default"; + v.compatible(systemIndexWarning); + })); assertBusy(() -> { try { assertThat(client().performRequest(getTasksIndex).getStatusLine().getStatusCode(), is(200)); diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java index 48920b3fdf3d2..0f1219d45a326 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -75,10 +75,11 @@ public void testSystemIndexAccessBlockedByDefault() throws Exception { } - // make sure the system index now exists (with allow_system_index_access flag) + // make sure the system index now exists assertBusy(() -> { Request searchRequest = new Request("GET", "/" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + "/_count"); - searchRequest.addParameter("allow_system_index_access", "true"); + searchRequest.setOptions(expectWarnings("this request accesses system indices: [" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + + "], but in a future major version, direct access to system indices will be prevented by default")); // Disallow no indices to cause an exception if the flag above doesn't work searchRequest.addParameter("allow_no_indices", "false"); @@ -91,9 +92,6 @@ public void testSystemIndexAccessBlockedByDefault() throws Exception { assertThat(responseMap.get("count"), equalTo(1)); }); - // now try without `allow_system_index_access` - assertDeprecationWarningOnAccess(SystemIndexTestPlugin.SYSTEM_INDEX_NAME, SystemIndexTestPlugin.SYSTEM_INDEX_NAME); - // And with a partial wildcard assertDeprecationWarningOnAccess(".test-*", SystemIndexTestPlugin.SYSTEM_INDEX_NAME); @@ -112,10 +110,9 @@ public void testSystemIndexAccessBlockedByDefault() throws Exception { { String expectedWarning = "this request accesses system indices: [" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + "], but in a " + "future major version, direct access to system indices will be prevented by default"; - RequestOptions expectWarningOptions = RequestOptions.DEFAULT.toBuilder().setWarningsHandler(expectedWarning::equals).build(); Request putDocDirectlyRequest = new Request("PUT", "/" + SystemIndexTestPlugin.SYSTEM_INDEX_NAME + "/_doc/43"); putDocDirectlyRequest.setJsonEntity("{\"some_field\": \"some_other_value\"}"); - putDocDirectlyRequest.setOptions(expectWarningOptions); + putDocDirectlyRequest.setOptions(expectWarnings(expectedWarning)); Response response = getRestClient().performRequest(putDocDirectlyRequest); assertThat(response.getStatusLine().getStatusCode(), equalTo(201)); } @@ -124,19 +121,23 @@ public void testSystemIndexAccessBlockedByDefault() throws Exception { private void assertDeprecationWarningOnAccess(String queryPattern, String warningIndexName) throws IOException { String expectedWarning = "this request accesses system indices: [" + warningIndexName + "], but in a " + "future major version, direct access to system indices will be prevented by default"; - RequestOptions expectWarningOptions = RequestOptions.DEFAULT.toBuilder() - .setWarningsHandler(w -> w.contains(expectedWarning) == false || w.size() != 1) - .build(); Request searchRequest = new Request("GET", "/" + queryPattern + randomFrom("/_count", "/_search")); searchRequest.setJsonEntity("{\"query\": {\"match\": {\"some_field\": \"some_value\"}}}"); // Disallow no indices to cause an exception if this resolves to zero indices, so that we're sure it resolved the index searchRequest.addParameter("allow_no_indices", "false"); - searchRequest.setOptions(expectWarningOptions); + searchRequest.setOptions(expectWarnings(expectedWarning)); Response response = getRestClient().performRequest(searchRequest); assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); } + private RequestOptions expectWarnings(String expectedWarning) { + return RequestOptions.DEFAULT.toBuilder() + .setWarningsHandler(w -> w.contains(expectedWarning) == false || w.size() != 1) + .build(); + } + + public static class SystemIndexTestPlugin extends Plugin implements SystemIndexPlugin { public static final String SYSTEM_INDEX_NAME = ".test-system-idx"; diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index d606c7c7bff6d..58e990c61a305 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -100,7 +100,6 @@ import static java.util.Collections.sort; import static java.util.Collections.unmodifiableList; -import static org.elasticsearch.rest.RestHandler.ALLOW_SYSTEM_INDEX_ADDED_VERSION; import static org.hamcrest.Matchers.anEmptyMap; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; @@ -646,13 +645,23 @@ private void wipeCluster() throws Exception { protected static void wipeAllIndices() throws IOException { boolean includeHidden = minimumNodeVersion().onOrAfter(Version.V_7_7_0); - boolean includeSystem = minimumNodeVersion().onOrAfter(ALLOW_SYSTEM_INDEX_ADDED_VERSION); try { final Request deleteRequest = new Request("DELETE", "*"); deleteRequest.addParameter("expand_wildcards", "open,closed" + (includeHidden ? ",hidden" : "")); - if (includeSystem) { - deleteRequest.addParameter("allow_system_index_access", "true"); - } + RequestOptions allowSystemIndexAccessWarningOptions = RequestOptions.DEFAULT.toBuilder() + .setWarningsHandler(warnings -> { + if (warnings.size() == 0) { + return false; + } else if (warnings.size() > 1) { + return true; + } + // We don't know exactly which indices we're cleaning up in advance, so just accept all system index access warnings. + final String warning = warnings.get(0); + final boolean isSystemIndexWarning = warning.contains("this request accesses system indices") + && warning.contains("but in a future major version, direct access to system indices will be prevented by default"); + return isSystemIndexWarning == false; + }).build(); + deleteRequest.setOptions(allowSystemIndexAccessWarningOptions); final Response response = adminClient().performRequest(deleteRequest); try (InputStream is = response.getEntity().getContent()) { assertTrue((boolean) XContentHelper.convertToMap(XContentType.JSON.xContent(), is, true).get("acknowledged")); diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java index 9255133ce0780..18cf0518291b8 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java @@ -7,6 +7,7 @@ import org.apache.http.util.EntityUtils; import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.common.bytes.BytesReference; @@ -69,11 +70,25 @@ protected Settings restClientSettings() { @After public void cleanUpData() throws Exception { new MlRestTestStateCleaner(logger, adminClient()).clearMlMetadata(); + RequestOptions allowSystemIndexAccessWarningOptions = RequestOptions.DEFAULT.toBuilder() + .setWarningsHandler(warnings -> { + if (warnings.isEmpty()) { + // There may not be an index to delete, in which case there's no warning + return false; + } else if (warnings.size() > 1) { + logger.warn("too many warnings: {}", warnings); + return true; + } + // We don't know exactly which indices we're cleaning up in advance, so just accept all system index access warnings. + final String warning = warnings.get(0); + final boolean isSystemIndexWarning = warning.contains("this request accesses system indices") + && warning.contains("but in a future major version, direct access to system indices will be prevented by default"); + return isSystemIndexWarning == false; + }).build(); final Request deleteInferenceRequest = new Request("DELETE", InferenceIndexConstants.INDEX_PATTERN); - deleteInferenceRequest.addParameter("allow_system_index_access", "true"); + deleteInferenceRequest.setOptions(allowSystemIndexAccessWarningOptions); client().performRequest(deleteInferenceRequest); final Request deleteStatsRequest = new Request("DELETE", MlStatsIndex.indexPattern()); - deleteStatsRequest.addParameter("allow_system_index_access", "true"); client().performRequest(deleteStatsRequest); Request loggingSettings = new Request("PUT", "_cluster/settings"); loggingSettings.setJsonEntity("" + diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java index 38e75662a6861..c04d2ec72e47a 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java @@ -43,7 +43,8 @@ public void testDeleteConfigurationLeftOver() throws IOException { final StringEntity entity = new StringEntity(Strings.toString(builder), ContentType.APPLICATION_JSON); Request req = new Request("PUT", TransformInternalIndexConstants.LATEST_INDEX_NAME + "/_doc/" + TransformConfig.documentId(fakeTransformName)); - req.addParameter("allow_system_index_access", "true"); + req.setOptions(expectWarnings("this request accesses system indices: [" + TransformInternalIndexConstants.LATEST_INDEX_NAME + + "], but in a future major version, direct access to system indices will be prevented by default")); req.setEntity(entity); client().performRequest(req); } diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java index 9ef5fa1a86316..6f84d358a7e67 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformInternalIndexIT.java @@ -77,14 +77,16 @@ public void testUpdateDeletesOldTransformConfig() throws Exception { + "\"frequency\":\"1s\"" + "}"; Request indexRequest = new Request("PUT", OLD_INDEX + "/_doc/" + TransformConfig.documentId(transformId)); - indexRequest.addParameter("allow_system_index_access", "true"); + indexRequest.setOptions(expectWarnings("this request accesses system indices: [" + OLD_INDEX + "], but in a future major " + + "version, direct access to system indices will be prevented by default")); indexRequest.addParameter("refresh", "true"); indexRequest.setJsonEntity(config); assertOK(client().performRequest(indexRequest)); { Request getRequest = new Request("GET", OLD_INDEX + "/_doc/" + TransformConfig.documentId(transformId)); - getRequest.addParameter("allow_system_index_access", "true"); + getRequest.setOptions(expectWarnings("this request accesses system indices: [" + OLD_INDEX + "], but in a future major " + + "version, direct access to system indices will be prevented by default")); Response getResponse = client().performRequest(getRequest); assertOK(getResponse); } @@ -103,7 +105,8 @@ public void testUpdateDeletesOldTransformConfig() throws Exception { // Old should now be gone { Request getRequest = new Request("GET", OLD_INDEX + "/_doc/" + TransformConfig.documentId(transformId)); - getRequest.addParameter("allow_system_index_access", "true"); + getRequest.setOptions(expectWarnings("this request accesses system indices: [" + OLD_INDEX + "], but in a future major " + + "version, direct access to system indices will be prevented by default")); try { Response getResponse = client().performRequest(getRequest); assertThat(getResponse.getStatusLine().getStatusCode(), equalTo(404)); @@ -116,7 +119,8 @@ public void testUpdateDeletesOldTransformConfig() throws Exception { // New should be here { Request getRequest = new Request("GET", CURRENT_INDEX + "/_doc/" + TransformConfig.documentId(transformId)); - getRequest.addParameter("allow_system_index_access", "true"); + getRequest.setOptions(expectWarnings("this request accesses system indices: [" + CURRENT_INDEX + "], but in a future major " + + "version, direct access to system indices will be prevented by default")); Response getResponse = client().performRequest(getRequest); assertOK(getResponse); } diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformRestTestCase.java b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformRestTestCase.java index da98d37e8d766..68a9de13f2f6b 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformRestTestCase.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformRestTestCase.java @@ -480,7 +480,8 @@ public void wipeTransforms() throws IOException { // the configuration index should be empty Request request = new Request("GET", TransformInternalIndexConstants.LATEST_INDEX_NAME + "/_search"); - request.addParameter("allow_system_index_access", "true"); + request.setOptions(expectWarnings("this request accesses system indices: [" + TransformInternalIndexConstants.LATEST_INDEX_NAME + + "], but in a future major version, direct access to system indices will be prevented by default")); try { Response searchResponse = adminClient().performRequest(request); Map searchResult = entityAsMap(searchResponse); diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformRobustnessIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformRobustnessIT.java index 3e28b006d8eae..d2014c79fc333 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformRobustnessIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformRobustnessIT.java @@ -118,7 +118,9 @@ private int getNumberOfTransformTasks() throws IOException { private void beEvilAndDeleteTheTransformIndex() throws IOException { final Request deleteRequest = new Request("DELETE", TransformInternalIndexConstants.LATEST_INDEX_NAME); - deleteRequest.addParameter("allow_system_index_access", "true"); + deleteRequest.setOptions(expectWarnings("this request accesses system indices: [" + + TransformInternalIndexConstants.LATEST_INDEX_NAME + "], but in a future major version, direct access to system indices will " + + "be prevented by default")); adminClient().performRequest(deleteRequest); } } diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java index bb366daa6a134..7ecde25a7fb32 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java @@ -62,7 +62,9 @@ public void testUsage() throws Exception { + ":" + TransformStoredDoc.NAME ); - statsExistsRequest.addParameter("allow_system_index_access", "true"); + statsExistsRequest.setOptions(expectWarnings("this request accesses system indices: [" + + TransformInternalIndexConstants.LATEST_INDEX_NAME + "], but in a future major version, direct access to system indices will " + + "be prevented by default")); // Verify that we have one stat document assertBusy(() -> { Map hasStatsMap = entityAsMap(client().performRequest(statsExistsRequest)); diff --git a/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherRestTestCase.java b/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherRestTestCase.java index 07eee59af7eb8..c44f1ed29bae3 100644 --- a/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherRestTestCase.java +++ b/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherRestTestCase.java @@ -73,7 +73,12 @@ public final void stopWatcher() throws Exception { Request deleteWatchesIndexRequest = new Request("DELETE", ".watches"); deleteWatchesIndexRequest.addParameter("ignore_unavailable", "true"); - deleteWatchesIndexRequest.addParameter("allow_system_index_access", "true"); + deleteWatchesIndexRequest.setOptions( + expectWarnings( + "this request accesses system indices: [.watches], but in a future major " + + "version, direct access to system indices will be prevented by default" + ) + ); ESRestTestCase.adminClient().performRequest(deleteWatchesIndexRequest); Request deleteWatchHistoryRequest = new Request("DELETE", ".watcher-history-*"); diff --git a/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java b/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java index 5a20f46ca1ec2..8a49f1e444a3e 100644 --- a/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java +++ b/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java @@ -105,7 +105,12 @@ public final void stopWatcher() throws Exception { private static void deleteWatcherIndices() throws IOException { Request deleteWatchesIndexRequest = new Request("DELETE", ".watches"); deleteWatchesIndexRequest.addParameter("ignore_unavailable", "true"); - deleteWatchesIndexRequest.addParameter("allow_system_index_access", "true"); + deleteWatchesIndexRequest.setOptions( + expectWarnings( + "this request accesses system indices: [.watches], but in a future major " + + "version, direct access to system indices will be prevented by default" + ) + ); ESRestTestCase.adminClient().performRequest(deleteWatchesIndexRequest); Request deleteWatchHistoryRequest = new Request("DELETE", ".watcher-history-*"); diff --git a/x-pack/plugin/watcher/qa/with-monitoring/src/javaRestTest/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java b/x-pack/plugin/watcher/qa/with-monitoring/src/javaRestTest/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java index 15e52ee8742e4..39e18f55909a5 100644 --- a/x-pack/plugin/watcher/qa/with-monitoring/src/javaRestTest/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java +++ b/x-pack/plugin/watcher/qa/with-monitoring/src/javaRestTest/java/org/elasticsearch/smoketest/MonitoringWithWatcherRestIT.java @@ -6,6 +6,7 @@ package org.elasticsearch.smoketest; import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.Response; import org.elasticsearch.common.Strings; import org.elasticsearch.test.rest.ESRestTestCase; @@ -40,7 +41,18 @@ public void cleanExporters() throws Exception { .endObject().endObject())); adminClient().performRequest(cleanupSettingsRequest); final Request deleteRequest = new Request("DELETE", "/.watch*"); - deleteRequest.addParameter("allow_system_index_access", "true"); + RequestOptions allowSystemIndexAccessWarningOptions = RequestOptions.DEFAULT.toBuilder() + .setWarningsHandler(warnings -> { + if (warnings.size() != 1) { + return true; + } + // We don't know exactly which indices we're cleaning up in advance, so just accept all system index access warnings. + final String warning = warnings.get(0); + final boolean isSystemIndexWarning = warning.contains("this request accesses system indices") + && warning.contains("but in a future major version, direct access to system indices will be prevented by default"); + return isSystemIndexWarning == false; + }).build(); + deleteRequest.setOptions(allowSystemIndexAccessWarningOptions); adminClient().performRequest(deleteRequest); } @@ -90,7 +102,8 @@ private void assertTotalWatchCount(int expectedWatches) throws Exception { assertBusy(() -> { refreshAllIndices(); final Request countRequest = new Request("POST", "/.watches/_count"); - countRequest.addParameter("allow_system_index_access", "true"); + countRequest.setOptions(expectWarnings("this request accesses system indices: [.watches], but in a future major " + + "version, direct access to system indices will be prevented by default")); ObjectPath path = ObjectPath.createFromResponse(client().performRequest(countRequest)); int count = path.evaluate("count"); assertThat(count, is(expectedWatches)); diff --git a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java index 5ad0f8072cc37..139ae375ca0a9 100644 --- a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java +++ b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java @@ -101,7 +101,8 @@ public void testSecurityNativeRealm() throws Exception { } else { waitForYellow(".security"); final Request getSettingsRequest = new Request("GET", "/.security/_settings/index.format"); - getSettingsRequest.addParameter("allow_system_index_access", "true"); + getSettingsRequest.setOptions(expectWarnings("this request accesses system indices: [.security], but in a future major " + + "version, direct access to system indices will be prevented by default")); Response settingsResponse = client().performRequest(getSettingsRequest); Map settingsResponseMap = entityAsMap(settingsResponse); logger.info("settings response map {}", settingsResponseMap); @@ -177,7 +178,8 @@ public void testWatcher() throws Exception { logger.info("checking that the Watches index is the correct version"); final Request getSettingsRequest = new Request("GET", "/.watches/_settings/index.format"); - getSettingsRequest.addParameter("allow_system_index_access", "true"); + getSettingsRequest.setOptions(expectWarnings("this request accesses system indices: [.watches], but in a future major " + + "version, direct access to system indices will be prevented by default")); Response settingsResponse = client().performRequest(getSettingsRequest); Map settingsResponseMap = entityAsMap(settingsResponse); logger.info("settings response map {}", settingsResponseMap); diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java index b0fafc2d40aef..00cfe0f4d4de5 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlMappingsUpgradeIT.java @@ -163,7 +163,8 @@ private void assertUpgradedConfigMappings() throws Exception { assertBusy(() -> { Request getMappings = new Request("GET", ".ml-config/_mappings"); - getMappings.addParameter("allow_system_index_access", "true"); + getMappings.setOptions(expectWarnings("this request accesses system indices: [.ml-config], but in a future major " + + "version, direct access to system indices will be prevented by default")); Response response = client().performRequest(getMappings); Map responseLevel = entityAsMap(response); From 08c2ec94e8056e8534fd6f904f4c505172278452 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 24 Sep 2020 11:43:10 -0600 Subject: [PATCH 113/161] Replace allow_system_index_access flags in yaml tests with warnings handlers --- .../test/delete_by_query/80_slices.yml | 10 ++++++++-- .../rest-api-spec/test/reindex/80_slices.yml | 10 ++++++++-- .../test/update_by_query/70_slices.yml | 10 ++++++++-- .../test/upgraded_cluster/10_basic.yml | 5 +++-- .../rest-api-spec/test/ml/calendar_crud.yml | 8 ++++++-- .../rest-api-spec/test/ml/custom_all_field.yml | 1 - .../resources/rest-api-spec/test/ml/filter_crud.yml | 9 +++++++-- .../rest-api-spec/test/ml/index_layout.yml | 10 ++++++++-- .../rest-api-spec/test/ml/inference_crud.yml | 4 +++- .../rest-api-spec/test/ml/inference_stats_crud.yml | 10 +++++++--- .../resources/rest-api-spec/test/users/10_basic.yml | 5 ++++- .../xpack/watcher/WatcherYamlSuiteTestCase.java | 13 +++++++------ .../test/watcher/ack_watch/10_basic.yml | 5 ++++- .../test/watcher/activate_watch/10_basic.yml | 8 ++++++-- .../test/watcher/delete_watch/10_basic.yml | 5 ++++- .../test/watcher/get_watch/10_basic.yml | 5 ++++- .../put_watch/80_put_get_watch_with_passwords.yml | 5 ++++- .../upgraded_cluster/80_transform_jobs_crud.yml | 5 ++++- 18 files changed, 95 insertions(+), 33 deletions(-) diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml index 550b9a3c2f767..9f45dcb07a4c1 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml @@ -62,6 +62,8 @@ --- "Multiple slices with wait_for_completion=false": + - skip: + features: warnings - do: index: index: test @@ -153,8 +155,9 @@ - do: indices.refresh: {} - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" search: - allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } @@ -166,6 +169,8 @@ --- "Multiple slices with rethrottle": + - skip: + features: warnings - do: index: index: test @@ -262,8 +267,9 @@ - do: indices.refresh: {} - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" search: - allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml index 6bd34b7d40629..4d4af5dd0a8d8 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml @@ -58,6 +58,8 @@ --- "Multiple slices with wait_for_completion=false": + - skip: + features: warnings - do: index: index: source @@ -162,8 +164,9 @@ - do: indices.refresh: {} - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" search: - allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } @@ -171,6 +174,8 @@ --- "Multiple slices with rethrottle": + - skip: + features: warnings - do: index: index: source @@ -275,8 +280,9 @@ - do: indices.refresh: {} - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" search: - allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml index 05cf20e4253a8..000693937ce7f 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml @@ -54,6 +54,8 @@ --- "Multiple slices with wait_for_completion=false": + - skip: + features: warnings - do: index: index: test @@ -145,14 +147,17 @@ - do: indices.refresh: {} - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" search: - allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } --- "Multiple slices with rethrottle": + - skip: + features: warnings - do: index: index: test @@ -249,8 +254,9 @@ - do: indices.refresh: {} - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" search: - allow_system_index_access: true rest_total_hits_as_int: true index: .tasks - match: { hits.total: 1 } diff --git a/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml index 67bd9de46e575..7a360fe8ae0a5 100644 --- a/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml +++ b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml @@ -87,11 +87,12 @@ --- "Find a task result record from the old cluster": - skip: - features: headers + features: headers,warnings - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" search: - allow_system_index_access: true rest_total_hits_as_int: true index: .tasks body: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/calendar_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/calendar_crud.yml index a359fd8579f6c..cfe0ead9e5040 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/calendar_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/calendar_crud.yml @@ -389,6 +389,8 @@ --- "Test delete calendar deletes events": + - skip: + features: warnings - do: ml.put_calendar: @@ -425,8 +427,9 @@ # Check the event from calendar 1 is deleted - do: + warnings: + - "this request accesses system indices: [.ml-meta], but in a future major version, direct access to system indices will be prevented by default" count: - allow_system_index_access: true index: .ml-meta body: query: @@ -437,8 +440,9 @@ - match: { count: 2 } - do: + warnings: + - "this request accesses system indices: [.ml-meta], but in a future major version, direct access to system indices will be prevented by default" count: - allow_system_index_access: true index: .ml-meta body: query: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml index fd3c1d601f135..f6bf53fb289db 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml @@ -148,7 +148,6 @@ setup: - do: search: - allow_system_index_access: true expand_wildcards: all rest_total_hits_as_int: true body: { query: { bool: { must: [ diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml index 63cbac7515b0c..3e92d7ca271aa 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml @@ -4,15 +4,17 @@ setup: features: - headers - allowed_warnings + - warnings - do: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser Content-Type: application/json allowed_warnings: - "index [.ml-meta] matches multiple legacy templates [.ml-meta, global], composable templates will only match a single template" + warnings: + - "this request accesses system indices: [.ml-meta], but in a future major version, direct access to system indices will be prevented by default" index: index: .ml-meta - allow_system_index_access: true id: filter_imposter-filter body: > { @@ -320,10 +322,13 @@ setup: --- "Test get all filter given index exists but no mapping for filter_id": + - skip: + features: warnings - do: + warnings: + - "this request accesses system indices: [.ml-meta], but in a future major version, direct access to system indices will be prevented by default" indices.delete: - allow_system_index_access: true index: ".ml-meta" - do: allowed_warnings: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml index f186889466cd5..606d872b9267f 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml @@ -3,6 +3,8 @@ setup: features: headers --- "Test CRUD on two jobs in shared index": + - skip: + features: warnings - do: ml.put_job: @@ -101,10 +103,11 @@ setup: - is_true: '' - do: + warnings: + - "this request accesses system indices: [.ml-state-write], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: - allow_system_index_access: true name: ".ml-state-write" - is_true: '' @@ -645,6 +648,8 @@ setup: --- "Test force close does not create state": + - skip: + features: warnings - do: headers: @@ -697,10 +702,11 @@ setup: - is_true: '' - do: + warnings: + - "this request accesses system indices: [.ml-state-write], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: - allow_system_index_access: true name: ".ml-state-write" - is_true: '' diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_crud.yml index 0b614ace6ca3d..562cc7e7df924 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_crud.yml @@ -3,16 +3,18 @@ setup: features: - headers - allowed_warnings + - warnings - do: allowed_warnings: - "index [.ml-inference-000003] matches multiple legacy templates [.ml-inference-000003, global], composable templates will only match a single template" + warnings: + - "this request accesses system indices: [.ml-inference-000003], but in a future major version, direct access to system indices will be prevented by default" headers: Content-Type: application/json Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: id: trained_model_metadata-a-regression-model-0 index: .ml-inference-000003 - allow_system_index_access: true body: model_id: "a-regression-model-0" doc_type: "trained_model_metadata" diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml index 311c73ca3b0f3..9d420c38bef68 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml @@ -3,13 +3,15 @@ setup: features: - headers - allowed_warnings + - warnings - do: allowed_warnings: - "index [.ml-inference-000003] matches multiple legacy templates [.ml-inference-000003, global], composable templates will only match a single template" + warnings: + - "this request accesses system indices: [.ml-inference-000003], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: - allow_system_index_access: true id: trained_model_config-a-unused-regression-model1-0 index: .ml-inference-000003 body: > @@ -24,10 +26,11 @@ setup: } - do: + warnings: + - "this request accesses system indices: [.ml-inference-000003], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: - allow_system_index_access: true id: trained_model_config-a-unused-regression-model-0 index: .ml-inference-000003 body: > @@ -41,10 +44,11 @@ setup: "doc_type": "trained_model_config" } - do: + warnings: + - "this request accesses system indices: [.ml-inference-000003], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser index: - allow_system_index_access: true id: trained_model_config-a-used-regression-model-0 index: .ml-inference-000003 body: > diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml index e560b1d5204c2..d2ddeeb02e939 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml @@ -107,6 +107,8 @@ teardown: } --- "Test put user with password hash": + - skip: + features: warnings # Mostly this chain of put_user , search index, set value is to work around the fact that the # rest tests treat anything with a leading "$" as a stashed value, and bcrypt passwords start with "$" @@ -122,8 +124,9 @@ teardown: } - do: + warnings: + - "this request accesses system indices: [.security], but in a future major version, direct access to system indices will be prevented by default" get: - allow_system_index_access: true index: .security id: user-bob - set: { _source.password: "hash" } diff --git a/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java b/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java index 8a49f1e444a3e..4f59c01c4dd10 100644 --- a/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java +++ b/x-pack/plugin/watcher/qa/common/src/main/java/org/elasticsearch/xpack/watcher/WatcherYamlSuiteTestCase.java @@ -8,6 +8,7 @@ import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; @@ -105,12 +106,12 @@ public final void stopWatcher() throws Exception { private static void deleteWatcherIndices() throws IOException { Request deleteWatchesIndexRequest = new Request("DELETE", ".watches"); deleteWatchesIndexRequest.addParameter("ignore_unavailable", "true"); - deleteWatchesIndexRequest.setOptions( - expectWarnings( - "this request accesses system indices: [.watches], but in a future major " - + "version, direct access to system indices will be prevented by default" - ) - ); + deleteWatchesIndexRequest.setOptions(RequestOptions.DEFAULT.toBuilder().setWarningsHandler(warnings -> { + final String expectedWaring = "this request accesses system indices: [.watches], but in a future major version, direct " + + "access to system indices will be prevented by default"; + // There might not be a warning if the .watches index doesn't exist + return (warnings.isEmpty() || warnings.get(0).equals(expectedWaring)) == false; + })); ESRestTestCase.adminClient().performRequest(deleteWatchesIndexRequest); Request deleteWatchHistoryRequest = new Request("DELETE", ".watcher-history-*"); diff --git a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/ack_watch/10_basic.yml b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/ack_watch/10_basic.yml index 9fa8c627394d3..3a3f962ce6f54 100644 --- a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/ack_watch/10_basic.yml +++ b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/ack_watch/10_basic.yml @@ -1,5 +1,7 @@ --- "Test ack watch api": + - skip: + features: warnings - do: cluster.health: wait_for_status: yellow @@ -44,9 +46,10 @@ - match: { "status.actions.test_index.ack.state" : "awaits_successful_execution" } - do: + warnings: + - "this request accesses system indices: [.watches], but in a future major version, direct access to system indices will be prevented by default" search: rest_total_hits_as_int: true - allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 1 } diff --git a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/activate_watch/10_basic.yml b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/activate_watch/10_basic.yml index ecf873857801e..015310babd29d 100644 --- a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/activate_watch/10_basic.yml +++ b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/activate_watch/10_basic.yml @@ -1,5 +1,7 @@ --- "Test activate watch api": + - skip: + features: warnings - do: cluster.health: wait_for_status: yellow @@ -48,9 +50,10 @@ - match: { status.state.active : false } - do: + warnings: + - "this request accesses system indices: [.watches], but in a future major version, direct access to system indices will be prevented by default" search: rest_total_hits_as_int: true - allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 1 } @@ -70,9 +73,10 @@ - match: { status.state.active : true } - do: + warnings: + - "this request accesses system indices: [.watches], but in a future major version, direct access to system indices will be prevented by default" search: rest_total_hits_as_int: true - allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 1 } diff --git a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/delete_watch/10_basic.yml b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/delete_watch/10_basic.yml index 032ea714425ec..1b999a5eabe35 100644 --- a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/delete_watch/10_basic.yml +++ b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/delete_watch/10_basic.yml @@ -13,6 +13,8 @@ teardown: --- "Test delete watch api": + - skip: + features: warnings - do: watcher.put_watch: id: "my_watch" @@ -52,9 +54,10 @@ teardown: - match: { found: true } - do: + warnings: + - "this request accesses system indices: [.watches], but in a future major version, direct access to system indices will be prevented by default" search: rest_total_hits_as_int: true - allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 0 } diff --git a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/get_watch/10_basic.yml b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/get_watch/10_basic.yml index 626adeebb5fac..913366853b974 100644 --- a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/get_watch/10_basic.yml +++ b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/get_watch/10_basic.yml @@ -13,6 +13,8 @@ teardown: --- "Test get watch api": + - skip: + features: warnings - do: watcher.put_watch: id: "my_watch" @@ -47,9 +49,10 @@ teardown: - match: { created: true } - do: + warnings: + - "this request accesses system indices: [.watches], but in a future major version, direct access to system indices will be prevented by default" search: rest_total_hits_as_int: true - allow_system_index_access: true index: .watches body: { "query": { "term": { "_id": "my_watch" } } } - match: { hits.total: 1 } diff --git a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml index a1cd025c72df3..b66579d7b044e 100644 --- a/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml +++ b/x-pack/plugin/watcher/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/watcher/put_watch/80_put_get_watch_with_passwords.yml @@ -117,6 +117,8 @@ setup: --- "Test putting a watch with a redacted password with old seq no returns an error": + - skip: + features: warnings # version 1 - do: watcher.put_watch: @@ -260,9 +262,10 @@ setup: } - do: + warnings: + - "this request accesses system indices: [.watches], but in a future major version, direct access to system indices will be prevented by default" search: rest_total_hits_as_int: true - allow_system_index_access: true index: .watches body: > { diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_transform_jobs_crud.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_transform_jobs_crud.yml index 44ff08ce1a2fd..9d2f27565d2ad 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_transform_jobs_crud.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_transform_jobs_crud.yml @@ -279,6 +279,8 @@ setup: --- "Test index mappings for latest internal index and audit index": + - skip: + features: warnings - do: transform.put_transform: transform_id: "upgraded-simple-transform" @@ -295,9 +297,10 @@ setup: - match: { acknowledged: true } - do: + warnings: + - "this request accesses system indices: [.transform-internal-005], but in a future major version, direct access to system indices will be prevented by default" indices.get_mapping: index: .transform-internal-005 - allow_system_index_access: true - match: { \.transform-internal-005.mappings.dynamic: "false" } - match: { \.transform-internal-005.mappings.properties.id.type: "keyword" } - do: From bb27000c40fb25c9cf0d4a6689ee5f05843dfb7e Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 24 Sep 2020 14:54:50 -0600 Subject: [PATCH 114/161] Remove a couple unnecessary warning checks (they might be back later) --- .../test/resources/rest-api-spec/test/ml/index_layout.yml | 8 -------- 1 file changed, 8 deletions(-) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml index 606d872b9267f..2b8a9063b4ad3 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml @@ -3,8 +3,6 @@ setup: features: headers --- "Test CRUD on two jobs in shared index": - - skip: - features: warnings - do: ml.put_job: @@ -103,8 +101,6 @@ setup: - is_true: '' - do: - warnings: - - "this request accesses system indices: [.ml-state-write], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: @@ -648,8 +644,6 @@ setup: --- "Test force close does not create state": - - skip: - features: warnings - do: headers: @@ -702,8 +696,6 @@ setup: - is_true: '' - do: - warnings: - - "this request accesses system indices: [.ml-state-write], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.exists_alias: From e3c4d816fb644621282cc1c8ae7110d9bc81bd68 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 24 Sep 2020 14:56:42 -0600 Subject: [PATCH 115/161] The current security index is `.security-7`, not just ``.security` --- .../src/test/resources/rest-api-spec/test/users/10_basic.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml index d2ddeeb02e939..dcee957b6c3fb 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/users/10_basic.yml @@ -125,7 +125,7 @@ teardown: - do: warnings: - - "this request accesses system indices: [.security], but in a future major version, direct access to system indices will be prevented by default" + - "this request accesses system indices: [.security-7], but in a future major version, direct access to system indices will be prevented by default" get: index: .security id: user-bob From 3ac9ecfb36c39c1e25256fdee6d8bcfff812a665 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 24 Sep 2020 15:11:15 -0600 Subject: [PATCH 116/161] Fix skip:feature block in YAML test --- .../rest-api-spec/test/upgraded_cluster/10_basic.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml index 7a360fe8ae0a5..e987413087ebb 100644 --- a/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml +++ b/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/10_basic.yml @@ -87,7 +87,9 @@ --- "Find a task result record from the old cluster": - skip: - features: headers,warnings + features: + - headers + - warnings - do: warnings: From d3f5cd544259670a5dff73bf1675c059cdf1b1f0 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 24 Sep 2020 15:11:47 -0600 Subject: [PATCH 117/161] Add missing warning handler for current version --- .../java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java index f9ad83a29d0ec..6c858f01aca6c 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java @@ -74,6 +74,7 @@ public void testSystemIndicesUpgrades() throws Exception { getTasksIndex.setOptions(expectVersionSpecificWarnings(v -> { final String systemIndexWarning = "this request accesses system indices: [.tasks], but in a future major version, direct " + "access to system indices will be prevented by default"; + v.current(systemIndexWarning); v.compatible(systemIndexWarning); })); assertBusy(() -> { From 3668a7ba46da72faf40a1f14ea69c479b9fab939 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 24 Sep 2020 15:16:24 -0600 Subject: [PATCH 118/161] Undo _all special case handling --- .../elasticsearch/http/SystemIndexRestIT.java | 12 ++-------- .../metadata/IndexNameExpressionResolver.java | 22 +------------------ .../IndexNameExpressionResolverTests.java | 5 ++++- .../security/authz/AuthorizationService.java | 16 +------------- .../authz/IndicesAndAliasesResolver.java | 6 +---- .../authz/IndicesAndAliasesResolverTests.java | 3 +-- 6 files changed, 10 insertions(+), 54 deletions(-) diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java index 0f1219d45a326..70934124b156e 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -95,16 +95,8 @@ public void testSystemIndexAccessBlockedByDefault() throws Exception { // And with a partial wildcard assertDeprecationWarningOnAccess(".test-*", SystemIndexTestPlugin.SYSTEM_INDEX_NAME); - // Check that a total wildcard does not trigger a warning - { - String queryPattern = randomFrom("*", "_all"); - Request searchRequest = new Request("GET", "/" + queryPattern + randomFrom("/_count", "/_search")); - searchRequest.setJsonEntity("{\"query\": {\"match\": {\"some_field\": \"some_value\"}}}"); - // Disallow no indices to cause an exception if this resolves to zero indices, so that we're sure it resolved the index - searchRequest.addParameter("allow_no_indices", "false"); - Response response = getRestClient().performRequest(searchRequest); - assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); - } + // And with a total wildcard + assertDeprecationWarningOnAccess(randomFrom("*", "_all"), SystemIndexTestPlugin.SYSTEM_INDEX_NAME); // Try to index a doc directly { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index a6e69cb4b014c..124abfc398c55 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -70,7 +70,6 @@ public class IndexNameExpressionResolver { public static final String EXCLUDED_DATA_STREAMS_KEY = "es.excluded_ds"; public static final String SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY = "_system_index_access_allowed"; - public static final String ORIGINAL_INDICES_HEADER_KEY = "_original_indices"; public static final Version SYSTEM_INDEX_ENFORCEMENT_VERSION = Version.V_8_0_0; private final DateMathExpressionResolver dateMathExpressionResolver = new DateMathExpressionResolver(); @@ -316,7 +315,7 @@ Index[] concreteIndices(Context context, String... indexExpressions) { } private void checkSystemIndexAccess(Context context, Metadata metadata, Set concreteIndices, String[] originalPatterns) { - if (context.isSystemIndexAccessAllowed() == false && shouldEmitDeprecationWarning(originalPatterns)) { + if (context.isSystemIndexAccessAllowed() == false) { final List resolvedSystemIndices = concreteIndices.stream() .map(metadata::index) .filter(IndexMetadata::isSystem) @@ -717,25 +716,6 @@ private boolean isSystemIndexAccessAllowed() { return Booleans.parseBoolean(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY), true); } - private boolean shouldEmitDeprecationWarning(String[] patterns) { - String[] checkPatterns = patterns; - - String originalPatterns = threadContext.getHeader(ORIGINAL_INDICES_HEADER_KEY); - if (originalPatterns != null) { - checkPatterns = originalPatterns.split(","); - } - - if (checkPatterns == null) { - return false; - } else if (1 != checkPatterns.length) { - return isSystemIndexAccessAllowed() == false; - } else { - final String pattern = checkPatterns[0]; - final boolean isAllPattern = pattern == null || Metadata.ALL.equals(pattern) || Regex.isMatchAllPattern(pattern); - return (isSystemIndexAccessAllowed() || isAllPattern) == false; - } - } - public static class Context { private final ClusterState state; 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 6423b4eda312c..b3120fa1e9b7c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -1867,10 +1867,13 @@ public void testFullWildcardSystemIndexResolutionDeprecated() { SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); - SearchRequest request = new SearchRequest(randomFrom("*")); + SearchRequest request = new SearchRequest(randomFrom("*", "_all")); List indexNames = resolveConcreteIndexNameList(state, request); assertThat(indexNames, containsInAnyOrder("some-other-index", ".ml-stuff", ".ml-meta", ".watches")); + assertWarnings("this request accesses system indices: [.ml-meta, .ml-stuff, .watches], but in a future major version, " + + "direct access to system indices will be prevented by default"); + } public void testSingleSystemIndexResolutionDeprecated() { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java index 5b041a8e61f52..6d7d4d896c009 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationService.java @@ -11,7 +11,6 @@ import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteRequest; -import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.StepListener; import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesAction; @@ -88,7 +87,6 @@ import java.util.function.Consumer; import static org.elasticsearch.action.support.ContextPreservingActionListener.wrapPreservingContext; -import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ORIGINAL_INDICES_HEADER_KEY; import static org.elasticsearch.common.Strings.collectionToCommaDelimitedString; import static org.elasticsearch.xpack.core.security.SecurityField.setting; import static org.elasticsearch.xpack.core.security.authz.AuthorizationServiceField.ACTION_SCOPE_AUTHORIZATION_KEYS; @@ -129,7 +127,7 @@ public AuthorizationService(Settings settings, CompositeRolesStore rolesStore, C IndexNameExpressionResolver resolver) { this.clusterService = clusterService; this.auditTrailService = auditTrailService; - this.indicesAndAliasesResolver = new IndicesAndAliasesResolver(settings, clusterService, resolver, threadPool.getThreadContext()); + this.indicesAndAliasesResolver = new IndicesAndAliasesResolver(settings, clusterService, resolver); this.authcFailureHandler = authcFailureHandler; this.threadContext = threadPool.getThreadContext(); this.anonymousUser = anonymousUser; @@ -283,7 +281,6 @@ private void authorizeAction(final RequestInfo requestInfo, final String request } })); }); - storeOriginalIndicesHeader(requestInfo); authzEngine.authorizeIndexAction(requestInfo, authzInfo, resolvedIndicesAsyncSupplier, metadata.getIndicesLookup(), wrapPreservingContext(new AuthorizationResultListener<>(result -> handleIndexActionAuthorizationResult(result, requestInfo, requestId, authzInfo, authzEngine, authorizedIndicesSupplier, @@ -296,17 +293,6 @@ private void authorizeAction(final RequestInfo requestInfo, final String request } } - private void storeOriginalIndicesHeader(RequestInfo requestInfo) { - // We only care about the indices specified in the first layer of the request, as that's what the user actually asked for. - if (requestInfo.getRequest() instanceof IndicesRequest && threadContext.getHeader(ORIGINAL_INDICES_HEADER_KEY) == null) { - IndicesRequest indicesRequest = (IndicesRequest) requestInfo.getRequest(); - threadContext.putHeader(ORIGINAL_INDICES_HEADER_KEY, - indicesRequest.indices() == null || indicesRequest.indices().length == 0 - ? Metadata.ALL - : String.join(",", indicesRequest.indices())); - } - } - private void handleIndexActionAuthorizationResult(final IndexAuthorizationResult result, final RequestInfo requestInfo, final String requestId, final AuthorizationInfo authzInfo, final AuthorizationEngine authzEngine, diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java index c05fb734c9bd1..c52d2c111bf47 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java @@ -25,7 +25,6 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.protocol.xpack.graph.GraphExploreRequest; import org.elasticsearch.transport.RemoteClusterAware; @@ -57,14 +56,11 @@ class IndicesAndAliasesResolver { private final IndexNameExpressionResolver nameExpressionResolver; private final IndexAbstractionResolver indexAbstractionResolver; private final RemoteClusterResolver remoteClusterResolver; - private final ThreadContext threadContext; - IndicesAndAliasesResolver(Settings settings, ClusterService clusterService, IndexNameExpressionResolver resolver, - ThreadContext threadContext) { + IndicesAndAliasesResolver(Settings settings, ClusterService clusterService, IndexNameExpressionResolver resolver) { this.nameExpressionResolver = resolver; this.indexAbstractionResolver = new IndexAbstractionResolver(resolver); this.remoteClusterResolver = new RemoteClusterResolver(settings, clusterService.getClusterSettings()); - this.threadContext = threadContext; } /** diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java index ff4510b272250..9c3d7b036c3e6 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java @@ -267,9 +267,8 @@ public void setup() { ClusterService clusterService = mock(ClusterService.class); when(clusterService.getClusterSettings()).thenReturn(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); - final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); defaultIndicesResolver = - new IndicesAndAliasesResolver(settings, clusterService, new IndexNameExpressionResolver(threadContext), threadContext); + new IndicesAndAliasesResolver(settings, clusterService, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY))); } public void testDashIndicesAreAllowedInShardLevelRequests() { From acc455ae0301541fc9fa182189167e6b8f3273d3 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 24 Sep 2020 16:06:49 -0600 Subject: [PATCH 119/161] Remove allow_system_index_access flag --- docs/reference/api-conventions.asciidoc | 11 +++-------- docs/reference/docs/delete-by-query.asciidoc | 6 ------ docs/reference/docs/update-by-query.asciidoc | 6 ------ docs/reference/index-modules/blocks.asciidoc | 6 ------ docs/reference/indices/alias-exists.asciidoc | 6 ------ .../reference/indices/apis/reload-analyzers.asciidoc | 6 ------ docs/reference/indices/clearcache.asciidoc | 6 ------ docs/reference/indices/close.asciidoc | 6 ------ docs/reference/indices/delete-index.asciidoc | 6 ------ docs/reference/indices/flush.asciidoc | 6 ------ docs/reference/indices/forcemerge.asciidoc | 6 ------ docs/reference/indices/get-alias.asciidoc | 6 ------ docs/reference/indices/get-field-mapping.asciidoc | 6 ------ docs/reference/indices/get-index.asciidoc | 6 ------ docs/reference/indices/get-mapping.asciidoc | 6 ------ docs/reference/indices/get-settings.asciidoc | 6 ------ docs/reference/indices/indices-exists.asciidoc | 6 ------ docs/reference/indices/open-close.asciidoc | 6 ------ docs/reference/indices/put-mapping.asciidoc | 6 ------ docs/reference/indices/refresh.asciidoc | 6 ------ docs/reference/indices/resolve.asciidoc | 6 ------ docs/reference/indices/segments.asciidoc | 6 ------ docs/reference/indices/shard-stores.asciidoc | 6 ------ docs/reference/indices/stats.asciidoc | 6 ------ docs/reference/indices/update-settings.asciidoc | 6 ------ docs/reference/rest-api/common-parms.asciidoc | 7 ------- docs/reference/search/count.asciidoc | 6 ------ docs/reference/search/field-caps.asciidoc | 6 ------ docs/reference/search/multi-search.asciidoc | 12 ------------ docs/reference/search/rank-eval.asciidoc | 6 ------ docs/reference/search/search-shards.asciidoc | 6 ------ docs/reference/search/search-template.asciidoc | 6 ------ docs/reference/search/search.asciidoc | 6 ------ docs/reference/search/validate.asciidoc | 6 ------ .../resources/rest-api-spec/api/cluster.health.json | 4 ---- .../src/main/resources/rest-api-spec/api/count.json | 4 ---- .../resources/rest-api-spec/api/delete_by_query.json | 4 ---- .../main/resources/rest-api-spec/api/field_caps.json | 4 ---- .../src/main/resources/rest-api-spec/api/get.json | 4 ---- .../src/main/resources/rest-api-spec/api/index.json | 4 ---- .../rest-api-spec/api/indices.add_block.json | 4 ---- .../rest-api-spec/api/indices.clear_cache.json | 4 ---- .../resources/rest-api-spec/api/indices.close.json | 4 ---- .../resources/rest-api-spec/api/indices.delete.json | 4 ---- .../resources/rest-api-spec/api/indices.exists.json | 4 ---- .../rest-api-spec/api/indices.exists_alias.json | 4 ---- .../rest-api-spec/api/indices.exists_type.json | 4 ---- .../resources/rest-api-spec/api/indices.flush.json | 4 ---- .../rest-api-spec/api/indices.forcemerge.json | 4 ---- .../resources/rest-api-spec/api/indices.get.json | 4 ---- .../rest-api-spec/api/indices.get_alias.json | 4 ---- .../rest-api-spec/api/indices.get_field_mapping.json | 4 ---- .../rest-api-spec/api/indices.get_mapping.json | 4 ---- .../rest-api-spec/api/indices.get_settings.json | 4 ---- .../resources/rest-api-spec/api/indices.open.json | 4 ---- .../rest-api-spec/api/indices.put_mapping.json | 4 ---- .../rest-api-spec/api/indices.put_settings.json | 4 ---- .../rest-api-spec/api/indices.resolve_index.json | 4 ---- .../rest-api-spec/api/indices.segments.json | 4 ---- .../rest-api-spec/api/indices.shard_stores.json | 4 ---- .../resources/rest-api-spec/api/indices.stats.json | 4 ---- .../resources/rest-api-spec/api/indices.upgrade.json | 4 ---- .../rest-api-spec/api/indices.validate_query.json | 4 ---- .../main/resources/rest-api-spec/api/rank_eval.json | 4 ---- .../src/main/resources/rest-api-spec/api/search.json | 4 ---- .../resources/rest-api-spec/api/search_shards.json | 4 ---- .../resources/rest-api-spec/api/update_by_query.json | 4 ---- .../java/org/elasticsearch/rest/RestController.java | 7 +------ .../java/org/elasticsearch/rest/RestHandler.java | 3 --- x-pack/docs/en/watcher/managing-watches.asciidoc | 4 ++-- x-pack/docs/en/watcher/troubleshooting.asciidoc | 8 ++++---- 71 files changed, 10 insertions(+), 360 deletions(-) diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index 2ac9912da9b9a..3bea76b3f2e6e 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -46,14 +46,9 @@ Some indices (hereafter "system indices") are used by various system modules and/or plugins to store state or configuration. These indices are not intended to be accessed directly, and accessing them directly is deprecated. In the next major version, access to these indices will be prevented -by default to prevent accidental operations. You can opt in to access to these -indices by using the following query string parameter: - -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] - -Using this parameter will prevent deprecation warnings in the current version, -and maintain the current behavior with deprecation warnings in the next major -version. +by default to prevent accidental operations that may cause problems with +Elasticsearch features which depend on the consistency of data in these +indices. Some multi-target APIs that can target indices also support the following query string parameter: diff --git a/docs/reference/docs/delete-by-query.asciidoc b/docs/reference/docs/delete-by-query.asciidoc index ea77fdc48e89a..cd5f0f6beaac0 100644 --- a/docs/reference/docs/delete-by-query.asciidoc +++ b/docs/reference/docs/delete-by-query.asciidoc @@ -184,12 +184,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=from] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index 9727027080f3f..69c9ded788174 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -179,12 +179,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=from] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/index-modules/blocks.asciidoc b/docs/reference/index-modules/blocks.asciidoc index f7aeb39493438..8431dc8049247 100644 --- a/docs/reference/index-modules/blocks.asciidoc +++ b/docs/reference/index-modules/blocks.asciidoc @@ -110,12 +110,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=timeoutparms] diff --git a/docs/reference/indices/alias-exists.asciidoc b/docs/reference/indices/alias-exists.asciidoc index 1e039a1443d94..2596fddef6b34 100644 --- a/docs/reference/indices/alias-exists.asciidoc +++ b/docs/reference/indices/alias-exists.asciidoc @@ -40,12 +40,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `all`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=local] diff --git a/docs/reference/indices/apis/reload-analyzers.asciidoc b/docs/reference/indices/apis/reload-analyzers.asciidoc index f8c272f9efc1e..eba55ff615bdc 100644 --- a/docs/reference/indices/apis/reload-analyzers.asciidoc +++ b/docs/reference/indices/apis/reload-analyzers.asciidoc @@ -85,12 +85,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/clearcache.asciidoc b/docs/reference/indices/clearcache.asciidoc index 5a281b560d66f..7cfae6775ce16 100644 --- a/docs/reference/indices/clearcache.asciidoc +++ b/docs/reference/indices/clearcache.asciidoc @@ -45,12 +45,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - `fielddata`:: + -- diff --git a/docs/reference/indices/close.asciidoc b/docs/reference/indices/close.asciidoc index 2d19eb6b3be0e..f008857d366d6 100644 --- a/docs/reference/indices/close.asciidoc +++ b/docs/reference/indices/close.asciidoc @@ -50,12 +50,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=wait_for_active_shards] diff --git a/docs/reference/indices/delete-index.asciidoc b/docs/reference/indices/delete-index.asciidoc index b8d241c236fba..19112da63d9f2 100644 --- a/docs/reference/indices/delete-index.asciidoc +++ b/docs/reference/indices/delete-index.asciidoc @@ -55,12 +55,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=timeoutparms] diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index 5fabeb7f28313..4f5d5e0638f73 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -72,12 +72,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - `force`:: + -- diff --git a/docs/reference/indices/forcemerge.asciidoc b/docs/reference/indices/forcemerge.asciidoc index ac2bba0ebca2e..86ac22ba28494 100644 --- a/docs/reference/indices/forcemerge.asciidoc +++ b/docs/reference/indices/forcemerge.asciidoc @@ -89,12 +89,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - `flush`:: (Optional, boolean) If `true`, diff --git a/docs/reference/indices/get-alias.asciidoc b/docs/reference/indices/get-alias.asciidoc index 19fb38fd2d721..f2cb5ebab5ad4 100644 --- a/docs/reference/indices/get-alias.asciidoc +++ b/docs/reference/indices/get-alias.asciidoc @@ -50,12 +50,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `all`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `true`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=local] diff --git a/docs/reference/indices/get-field-mapping.asciidoc b/docs/reference/indices/get-field-mapping.asciidoc index a38e1af55fd4b..9f7626703cbdc 100644 --- a/docs/reference/indices/get-field-mapping.asciidoc +++ b/docs/reference/indices/get-field-mapping.asciidoc @@ -49,12 +49,6 @@ Defaults to `true`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] `include_defaults`:: diff --git a/docs/reference/indices/get-index.asciidoc b/docs/reference/indices/get-index.asciidoc index d46802114dc66..3087790b6ecc2 100644 --- a/docs/reference/indices/get-index.asciidoc +++ b/docs/reference/indices/get-index.asciidoc @@ -42,12 +42,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=include-defaults] diff --git a/docs/reference/indices/get-mapping.asciidoc b/docs/reference/indices/get-mapping.asciidoc index 528e132751b79..df71e073704ae 100644 --- a/docs/reference/indices/get-mapping.asciidoc +++ b/docs/reference/indices/get-mapping.asciidoc @@ -44,12 +44,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=local] diff --git a/docs/reference/indices/get-settings.asciidoc b/docs/reference/indices/get-settings.asciidoc index ded5cfd93ebe5..f655a7ff18a01 100644 --- a/docs/reference/indices/get-settings.asciidoc +++ b/docs/reference/indices/get-settings.asciidoc @@ -49,12 +49,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `all`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=include-defaults] diff --git a/docs/reference/indices/indices-exists.asciidoc b/docs/reference/indices/indices-exists.asciidoc index c1f3bf04eb3bb..6b9e9789bb38a 100644 --- a/docs/reference/indices/indices-exists.asciidoc +++ b/docs/reference/indices/indices-exists.asciidoc @@ -41,12 +41,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=include-defaults] diff --git a/docs/reference/indices/open-close.asciidoc b/docs/reference/indices/open-close.asciidoc index eb3c128a4d2dd..1490d3895d442 100644 --- a/docs/reference/indices/open-close.asciidoc +++ b/docs/reference/indices/open-close.asciidoc @@ -106,12 +106,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `closed`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=wait_for_active_shards] diff --git a/docs/reference/indices/put-mapping.asciidoc b/docs/reference/indices/put-mapping.asciidoc index 6728ae021c421..d7082f2db1125 100644 --- a/docs/reference/indices/put-mapping.asciidoc +++ b/docs/reference/indices/put-mapping.asciidoc @@ -53,12 +53,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=timeoutparms] diff --git a/docs/reference/indices/refresh.asciidoc b/docs/reference/indices/refresh.asciidoc index 7063c79ddd57e..98fbd5268aef1 100644 --- a/docs/reference/indices/refresh.asciidoc +++ b/docs/reference/indices/refresh.asciidoc @@ -84,12 +84,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `true`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/indices/resolve.asciidoc b/docs/reference/indices/resolve.asciidoc index 218856f77f9b7..3f0a5978f114b 100644 --- a/docs/reference/indices/resolve.asciidoc +++ b/docs/reference/indices/resolve.asciidoc @@ -76,12 +76,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - [[resolve-index-api-example]] ==== {api-examples-title} diff --git a/docs/reference/indices/segments.asciidoc b/docs/reference/indices/segments.asciidoc index c5c6bad3ef5c5..5fc0a9799a8b4 100644 --- a/docs/reference/indices/segments.asciidoc +++ b/docs/reference/indices/segments.asciidoc @@ -46,12 +46,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] `verbose`:: diff --git a/docs/reference/indices/shard-stores.asciidoc b/docs/reference/indices/shard-stores.asciidoc index e4cf7e7ce05bc..88e8f248a5317 100644 --- a/docs/reference/indices/shard-stores.asciidoc +++ b/docs/reference/indices/shard-stores.asciidoc @@ -64,12 +64,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] `status`:: diff --git a/docs/reference/indices/stats.asciidoc b/docs/reference/indices/stats.asciidoc index bf36c304e3c34..6352da9cc34e4 100644 --- a/docs/reference/indices/stats.asciidoc +++ b/docs/reference/indices/stats.asciidoc @@ -71,12 +71,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=fields] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=completion-fields] diff --git a/docs/reference/indices/update-settings.asciidoc b/docs/reference/indices/update-settings.asciidoc index e5ccb9bb335c3..d0b4fef89587e 100644 --- a/docs/reference/indices/update-settings.asciidoc +++ b/docs/reference/indices/update-settings.asciidoc @@ -50,12 +50,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=flat-settings] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/rest-api/common-parms.asciidoc b/docs/reference/rest-api/common-parms.asciidoc index cb93b678799ed..20ca52ceb05f9 100644 --- a/docs/reference/rest-api/common-parms.asciidoc +++ b/docs/reference/rest-api/common-parms.asciidoc @@ -228,13 +228,6 @@ Wildcard expressions are not accepted. -- end::expand-wildcards[] -tag::allow-system-index-access[] -`allow_system_index_access`:: -(Optional, boolean) If `true`, allows this request to access system indices -directly. This option should be used with caution, as system indices are not -intended to be accessed directly. -end::allow-system-index-access[] - tag::exponential-avg-checkpoint-duration-ms[] Exponential moving average of the duration of the checkpoint, in milliseconds. end::exponential-avg-checkpoint-duration-ms[] diff --git a/docs/reference/search/count.asciidoc b/docs/reference/search/count.asciidoc index 16e3cee2cc10b..07bb447c42522 100644 --- a/docs/reference/search/count.asciidoc +++ b/docs/reference/search/count.asciidoc @@ -66,12 +66,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=ignore_throttled] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/search/field-caps.asciidoc b/docs/reference/search/field-caps.asciidoc index bc8c259f66a1c..9d352273999ce 100644 --- a/docs/reference/search/field-caps.asciidoc +++ b/docs/reference/search/field-caps.asciidoc @@ -65,12 +65,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. -- -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] `include_unmapped`:: diff --git a/docs/reference/search/multi-search.asciidoc b/docs/reference/search/multi-search.asciidoc index 968f22a0075d1..a1da1ad5a7a4c 100644 --- a/docs/reference/search/multi-search.asciidoc +++ b/docs/reference/search/multi-search.asciidoc @@ -78,12 +78,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - `ignore_throttled`:: (Optional, boolean) If `true`, concrete, expanded or aliased indices are ignored when frozen. @@ -189,12 +183,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - `ignore_unavailable`::: (Optional, boolean) If `true`, documents from missing or closed indices are not included in the response. Defaults to `false`. diff --git a/docs/reference/search/rank-eval.asciidoc b/docs/reference/search/rank-eval.asciidoc index 659ce1560c776..2dfd6db9b5cbb 100644 --- a/docs/reference/search/rank-eval.asciidoc +++ b/docs/reference/search/rank-eval.asciidoc @@ -85,12 +85,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. -- -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] diff --git a/docs/reference/search/search-shards.asciidoc b/docs/reference/search/search-shards.asciidoc index f1550f78c2028..f44e129c0def6 100644 --- a/docs/reference/search/search-shards.asciidoc +++ b/docs/reference/search/search-shards.asciidoc @@ -44,12 +44,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] Defaults to `open`. -- -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailable] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=local] diff --git a/docs/reference/search/search-template.asciidoc b/docs/reference/search/search-template.asciidoc index a864a5c41dd5f..abc8689e936cb 100644 --- a/docs/reference/search/search-template.asciidoc +++ b/docs/reference/search/search-template.asciidoc @@ -62,12 +62,6 @@ Defaults to `true`. include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - `explain`:: (Optional, boolean) If `true`, the response includes additional details about score computation as part of a hit. Defaults to `false`. diff --git a/docs/reference/search/search.asciidoc b/docs/reference/search/search.asciidoc index 7ecaa1ea7ecf0..562bff49fef40 100644 --- a/docs/reference/search/search.asciidoc +++ b/docs/reference/search/search.asciidoc @@ -83,12 +83,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] + Defaults to `open`. -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - `explain`:: (Optional, boolean) If `true`, returns detailed information about score computation as part of a hit. Defaults to `false`. diff --git a/docs/reference/search/validate.asciidoc b/docs/reference/search/validate.asciidoc index b98bb5f6cc7b7..ba7cb0ba1986b 100644 --- a/docs/reference/search/validate.asciidoc +++ b/docs/reference/search/validate.asciidoc @@ -59,12 +59,6 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=df] include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=expand-wildcards] -include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=allow-system-index-access] -+ --- -Defaults to `false`. --- - `explain`:: (Optional, boolean) If `true`, the response returns detailed information if an error has occurred. Defaults to `false`. diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json index 09adf14085989..894b141f2f3b3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cluster.health.json @@ -98,10 +98,6 @@ "red" ], "description":"Wait until cluster is in a specific state" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json index cee111131e1e3..93a450afff88a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/count.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/count.json @@ -98,10 +98,6 @@ "terminate_after":{ "type":"number", "description":"The maximum count for each shard, upon reaching which the query execution will terminate early" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json index 27df42cbdf575..8641c18382dfc 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete_by_query.json @@ -176,10 +176,6 @@ "type":"number|string", "default":1, "description":"The number of slices this task should be divided into. Defaults to 1, meaning the task isn't sliced into subtasks. Can be set to `auto`." - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json index 841b1cb190cf9..20a87c7da8883 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json @@ -58,10 +58,6 @@ "type":"boolean", "default":false, "description":"Indicates whether unmapped fields should be included in the response." - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json index bb2d8212af0cf..36d08c0313bde 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/get.json @@ -70,10 +70,6 @@ "external_gte" ], "description":"Specific version type" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json index 3da3a66ab8a1b..b4865403331b0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json @@ -96,10 +96,6 @@ "require_alias": { "type": "boolean", "description": "When true, requires destination to be an alias. Default is false" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.add_block.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.add_block.json index bf410959be08a..7389fb1322824 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.add_block.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.add_block.json @@ -53,10 +53,6 @@ ], "default":"open", "description":"Whether to expand wildcard expression to concrete indices that are open, closed or both." - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json index ff23fe4eba0a9..64c10a520c7c4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.clear_cache.json @@ -67,10 +67,6 @@ "request":{ "type":"boolean", "description":"Clear request cache" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json index 71759a2a38e90..f26c8e77a06a6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.close.json @@ -53,10 +53,6 @@ "wait_for_active_shards":{ "type":"string", "description":"Sets the number of active shards to wait for before the operation returns." - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json index 243953b364204..53fdf44bb36a1 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.delete.json @@ -49,10 +49,6 @@ ], "default":"open", "description":"Whether wildcard expressions should get expanded to open or closed indices (default: open)" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json index c78ef2ab50512..7539f44a81eed 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists.json @@ -54,10 +54,6 @@ "type":"boolean", "description":"Whether to return all default setting for each of the indices.", "default":false - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json index 9cd4a9ce1a9c0..66e5ce92cbbe5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_alias.json @@ -61,10 +61,6 @@ "local":{ "type":"boolean", "description":"Return local information, do not retrieve the state from master node (default: false)" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json index 84357c8782d09..c854d0e8fd841 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.exists_type.json @@ -49,10 +49,6 @@ "local":{ "type":"boolean", "description":"Return local information, do not retrieve the state from master node (default: false)" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json index 276460f43b4b0..35138b920466f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.flush.json @@ -57,10 +57,6 @@ ], "default":"open", "description":"Whether to expand wildcard expression to concrete indices that are open, closed or both." - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json index fd8dca41045b2..6036b75bb83e4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.forcemerge.json @@ -59,10 +59,6 @@ "only_expunge_deletes":{ "type":"boolean", "description":"Specify whether the operation should only expunge deleted documents" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json index 455c73a589359..90a1274ecb059 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json @@ -58,10 +58,6 @@ "master_timeout":{ "type":"time", "description":"Specify timeout for connection to master" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json index 5eb8049a05750..5bfdd985a0a85 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_alias.json @@ -79,10 +79,6 @@ "local":{ "type":"boolean", "description":"Return local information, do not retrieve the state from master node (default: false)" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json index 29488591a9e19..0e71b6d395777 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json @@ -65,10 +65,6 @@ "local":{ "type":"boolean", "description":"Return local information, do not retrieve the state from master node (default: false)" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json index 559fc0203c297..24fd668069697 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json @@ -59,10 +59,6 @@ "version":"7.8.0", "description":"This parameter is a no-op and field mappings are always retrieved locally." } - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json index 99872624f3425..68e325446d3dc 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_settings.json @@ -92,10 +92,6 @@ "type":"boolean", "description":"Whether to return all default setting for each of the indices.", "default":false - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json index 98793b6ae2eb6..1dab468ce4ff4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.open.json @@ -53,10 +53,6 @@ "wait_for_active_shards":{ "type":"string", "description":"Sets the number of active shards to wait for before the operation returns." - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json index 52e0f0215a3bf..451cbccd8d329 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json @@ -55,10 +55,6 @@ "type":"boolean", "default":false, "description":"When true, applies mappings only to the write index of an alias or data stream" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json index 736b05618b7bb..66fe23bab8ba2 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_settings.json @@ -63,10 +63,6 @@ "flat_settings":{ "type":"boolean", "description":"Return settings in flat format (default: false)" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.resolve_index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.resolve_index.json index dd72c970c2ed4..41d609818dbc8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.resolve_index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.resolve_index.json @@ -33,10 +33,6 @@ ], "default":"open", "description":"Whether wildcard expressions should get expanded to open or closed indices (default: open)" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json index db4574b974cb4..83430a9a85600 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.segments.json @@ -52,10 +52,6 @@ "type":"boolean", "description":"Includes detailed memory usage by Lucene.", "default":false - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json index 86896bdca1a8a..7e48e99916171 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.shard_stores.json @@ -57,10 +57,6 @@ ], "default":"open", "description":"Whether to expand wildcard expression to concrete indices that are open, closed or both." - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json index 4576fb9fae848..2a3659dfb892a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.stats.json @@ -150,10 +150,6 @@ "type":"boolean", "description":"If set to false stats will also collected from closed indices if explicitly specified or if expand_wildcards expands to closed indices", "default":true - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json index 2cf232e28306e..406fbacda307c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.upgrade.json @@ -63,10 +63,6 @@ "only_ancient_segments":{ "type":"boolean", "description":"If true, only ancient (an older Lucene major release) segments will be upgraded" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json index 1d241fcdeffe5..3becec003a9e6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.validate_query.json @@ -112,10 +112,6 @@ "all_shards":{ "type":"boolean", "description":"Execute validation on all shards instead of one random shard per index" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json b/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json index 1e680d022f232..eadf240192394 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/rank_eval.json @@ -57,10 +57,6 @@ "dfs_query_then_fetch" ], "description":"Search operation type" - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json index 67c86c0ae23c3..03ca18705817e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search.json @@ -225,10 +225,6 @@ "type":"boolean", "description":"Indicates whether hits.total should be rendered as an integer or an object in the rest search response", "default":false - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json b/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json index 47157edba4ec7..74b7055b4c4b0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/search_shards.json @@ -61,10 +61,6 @@ ], "default":"open", "description":"Whether to expand wildcard expression to concrete indices that are open, closed or both." - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json index dbd5773ead331..86fa8a60db43b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/update_by_query.json @@ -184,10 +184,6 @@ "type":"number|string", "default":1, "description":"The number of slices this task should be divided into. Defaults to 1, meaning the task isn't sliced into subtasks. Can be set to `auto`." - }, - "allow_system_index_access": { - "type":"boolean", - "description":"Indicates whether this request should be allowed to access system indices" } }, "body":{ diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index 2c02dd9b75574..4c7097aa6a93b 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -24,7 +24,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.client.node.NodeClient; -import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; @@ -56,7 +55,6 @@ import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY; import static org.elasticsearch.rest.BytesRestResponse.TEXT_CONTENT_TYPE; -import static org.elasticsearch.rest.RestHandler.ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER; import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; import static org.elasticsearch.rest.RestStatus.INTERNAL_SERVER_ERROR; import static org.elasticsearch.rest.RestStatus.METHOD_NOT_ALLOWED; @@ -250,10 +248,7 @@ private void dispatchRequest(RestRequest request, RestChannel channel, RestHandl request.ensureSafeBuffers(); } if (handler.allowSystemIndexAccessByDefault() == false) { - final String allowSystemIndexParameter = request.param(ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER); - if (Booleans.parseBoolean(allowSystemIndexParameter, false) == false) { - threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); - } + threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); } handler.handleRequest(request, responseChannel, client); diff --git a/server/src/main/java/org/elasticsearch/rest/RestHandler.java b/server/src/main/java/org/elasticsearch/rest/RestHandler.java index 57155764e4731..90faea40af048 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/RestHandler.java @@ -33,9 +33,6 @@ @FunctionalInterface public interface RestHandler { - Version ALLOW_SYSTEM_INDEX_ADDED_VERSION = Version.V_8_0_0; - String ALLOW_SYSTEM_INDEX_ACCESS_REST_PARAMETER = "allow_system_index_access"; - /** * Handles a rest request. * @param request The request to handle diff --git a/x-pack/docs/en/watcher/managing-watches.asciidoc b/x-pack/docs/en/watcher/managing-watches.asciidoc index befd640673dac..828fe8ab0b3b6 100644 --- a/x-pack/docs/en/watcher/managing-watches.asciidoc +++ b/x-pack/docs/en/watcher/managing-watches.asciidoc @@ -28,9 +28,9 @@ For example, the following returns the first 100 watches: [source,console] -------------------------------------------------- -GET .watches/_search?allow_system_index_access=true +GET .watches/_search { "size" : 100 } -------------------------------------------------- -// TEST[setup:my_active_watch] +// TEST[skip:deprecation warning] diff --git a/x-pack/docs/en/watcher/troubleshooting.asciidoc b/x-pack/docs/en/watcher/troubleshooting.asciidoc index 1f99bd5388a17..352884298fdb0 100644 --- a/x-pack/docs/en/watcher/troubleshooting.asciidoc +++ b/x-pack/docs/en/watcher/troubleshooting.asciidoc @@ -16,9 +16,9 @@ do that by submitting the following request: [source,console] -------------------------------------------------- -GET .watches/_mapping?allow_system_index_access=true +GET .watches/_mapping -------------------------------------------------- -// TEST[setup:my_active_watch] +// TEST[skip:deprecation warning] If the index mappings are missing, follow these steps to restore the correct mappings: @@ -31,9 +31,9 @@ mappings: -- [source,console] -------------------------------------------------- -DELETE .watches?allow_system_index_access=true +DELETE .watches -------------------------------------------------- -// TEST[skip:index deletion] +// TEST[skip:index deletion and deprecation warning] -- . Disable direct access to the `.watches` index: .. Stop the Elasticsearch node. From aa48e2028b91f891a3231556040542c1d4491365 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 24 Sep 2020 16:15:45 -0600 Subject: [PATCH 120/161] More `.security` -> `.security-7` in expected warnings --- .../org/elasticsearch/xpack/restart/FullClusterRestartIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java index 139ae375ca0a9..fa7d8632fba4d 100644 --- a/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java +++ b/x-pack/qa/full-cluster-restart/src/test/java/org/elasticsearch/xpack/restart/FullClusterRestartIT.java @@ -101,7 +101,7 @@ public void testSecurityNativeRealm() throws Exception { } else { waitForYellow(".security"); final Request getSettingsRequest = new Request("GET", "/.security/_settings/index.format"); - getSettingsRequest.setOptions(expectWarnings("this request accesses system indices: [.security], but in a future major " + + getSettingsRequest.setOptions(expectWarnings("this request accesses system indices: [.security-7], but in a future major " + "version, direct access to system indices will be prevented by default")); Response settingsResponse = client().performRequest(getSettingsRequest); Map settingsResponseMap = entityAsMap(settingsResponse); From 04b7e4ba752f2131617547a1b2d8222770e574f1 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 24 Sep 2020 16:18:35 -0600 Subject: [PATCH 121/161] Unused import --- server/src/main/java/org/elasticsearch/rest/RestHandler.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/rest/RestHandler.java b/server/src/main/java/org/elasticsearch/rest/RestHandler.java index 90faea40af048..054c618876314 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/RestHandler.java @@ -19,7 +19,6 @@ package org.elasticsearch.rest; -import org.elasticsearch.Version; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.rest.RestRequest.Method; From e9d533268df72042e7861710194c23607c56cf77 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 30 Sep 2020 13:30:50 -0600 Subject: [PATCH 122/161] Work around _all resolution in Get Aliases API --- .../alias/get/TransportGetAliasesAction.java | 41 +++++++- .../get/TransportGetAliasesActionTests.java | 97 ++++++++++++++++++- 2 files changed, 130 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java index 4fbf8e84e62bd..c0e867a939c4c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java @@ -25,20 +25,26 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.AliasMetadata; +import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.logging.DeprecationLogger; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; +import java.util.Iterator; import java.util.List; public class TransportGetAliasesAction extends TransportMasterNodeReadAction { + private static DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(TransportGetAliasesAction.class); @Inject public TransportGetAliasesAction(TransportService transportService, ClusterService clusterService, @@ -56,8 +62,9 @@ protected String executor() { @Override protected ClusterBlockException checkBlock(GetAliasesRequest request, ClusterState state) { + // Resolve with system index access since we're just checking blocks return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, - indexNameExpressionResolver.concreteIndexNames(state, request)); + indexNameExpressionResolver.concreteIndexNamesWithSystemIndexAccess(state, request)); } @Override @@ -67,16 +74,22 @@ protected GetAliasesResponse read(StreamInput in) throws IOException { @Override protected void masterOperation(Task task, GetAliasesRequest request, ClusterState state, ActionListener listener) { - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); + String[] concreteIndices; + // Switch to a context which will drop any deprecation warnings, because there may be indices resolved here which are not + // returned in the final response. We'll add warnings back later if necessary in checkSystemIndexAccess. + try (ThreadContext.StoredContext ignore = threadPool.getThreadContext().newStoredContext(false)) { + concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); + } ImmutableOpenMap> aliases = state.metadata().findAliases(request, concreteIndices); - listener.onResponse(new GetAliasesResponse(postProcess(request, concreteIndices, aliases))); + listener.onResponse(new GetAliasesResponse(postProcess(request, concreteIndices, aliases, state))); } /** * Fills alias result with empty entries for requested indices when no specific aliases were requested. */ static ImmutableOpenMap> postProcess(GetAliasesRequest request, String[] concreteIndices, - ImmutableOpenMap> aliases) { + ImmutableOpenMap> aliases, + ClusterState state) { boolean noAliasesSpecified = request.getOriginalAliases() == null || request.getOriginalAliases().length == 0; ImmutableOpenMap.Builder> mapBuilder = ImmutableOpenMap.builder(aliases); for (String index : concreteIndices) { @@ -85,7 +98,25 @@ static ImmutableOpenMap> postProcess(GetAliasesReque assert previous == null; } } - return mapBuilder.build(); + final ImmutableOpenMap> finalResponse = mapBuilder.build(); + checkSystemIndexAccess(state, finalResponse); + return finalResponse; + } + + private static void checkSystemIndexAccess(ClusterState state, ImmutableOpenMap> aliasesMap) { + List systemIndices = new ArrayList<>(); + for (Iterator it = aliasesMap.keysIt(); it.hasNext(); ) { + String indexName = it.next(); + IndexMetadata index = state.metadata().index(indexName); + if (index != null && index.isSystem()) { + systemIndices.add(indexName); + } + } + if (systemIndices.isEmpty() == false) { + deprecationLogger.deprecate("open_system_index_access", + "this request accesses system indices: {}, but in a future major version, direct access to system " + + "indices will be prevented by default", systemIndices); + } } } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java index 967d94c15d6df..9659fd4523c82 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java @@ -18,7 +18,11 @@ */ package org.elasticsearch.action.admin.indices.alias.get; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.AliasMetadata; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.test.ESTestCase; @@ -35,7 +39,7 @@ public void testPostProcess() { .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) .build(); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases); + TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE); assertThat(result.size(), equalTo(3)); assertThat(result.get("a").size(), equalTo(0)); assertThat(result.get("b").size(), equalTo(1)); @@ -46,7 +50,7 @@ public void testPostProcess() { aliases = ImmutableOpenMap.>builder() .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) .build(); - result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases); + result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE); assertThat(result.size(), equalTo(3)); assertThat(result.get("a").size(), equalTo(0)); assertThat(result.get("b").size(), equalTo(1)); @@ -56,9 +60,96 @@ public void testPostProcess() { aliases = ImmutableOpenMap.>builder() .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) .build(); - result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases); + result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE); assertThat(result.size(), equalTo(1)); assertThat(result.get("b").size(), equalTo(1)); } + public void testDeprecationWarningEmittedForTotalWildcard() { + ClusterState state = systemIndexTestClusterState(); + + GetAliasesRequest request = new GetAliasesRequest(); + ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() + .fPut(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())) + .fPut("c", Collections.singletonList(new AliasMetadata.Builder("d").build())) + .build(); + final String[] concreteIndices = {"a", ".b", "c"}; + assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); + ImmutableOpenMap> result = + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state); + assertThat(result.size(), equalTo(3)); + assertThat(result.get("a").size(), equalTo(0)); + assertThat(result.get(".b").size(), equalTo(1)); + assertThat(result.get("c").size(), equalTo(1)); + assertWarnings("this request accesses system indices: [.b], but in a future major version, direct access to system " + + "indices will be prevented by default"); + } + + public void testDeprecationWarningEmittedWhenSystemIndexIsRequested() { + ClusterState state = systemIndexTestClusterState(); + + GetAliasesRequest request = new GetAliasesRequest(); + request.indices(".b"); + ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() + .fPut(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())) + .build(); + final String[] concreteIndices = {".b"}; + assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); + ImmutableOpenMap> result = + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state); + assertThat(result.size(), equalTo(1)); + assertThat(result.get(".b").size(), equalTo(1)); + assertWarnings("this request accesses system indices: [.b], but in a future major version, direct access to system " + + "indices will be prevented by default"); + } + + public void testDeprecationWarningEmittedWhenSystemIndexIsRequestedByAlias() { + ClusterState state = systemIndexTestClusterState(); + + GetAliasesRequest request = new GetAliasesRequest(".y"); + ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() + .fPut(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())) + .build(); + final String[] concreteIndices = {"a", ".b", "c"}; + assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); + ImmutableOpenMap> result = + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state); + assertThat(result.size(), equalTo(1)); + assertThat(result.get(".b").size(), equalTo(1)); + assertWarnings("this request accesses system indices: [.b], but in a future major version, direct access to system " + + "indices will be prevented by default"); + } + + /** + * Ensures that deprecation warnings are not emitted when + */ + public void testDeprecationWarningNotEmittedWhenOnlyNonsystemIndexRequested() { + ClusterState state = systemIndexTestClusterState(); + + GetAliasesRequest request = new GetAliasesRequest(); + request.indices("c"); + ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() + .fPut("c", Collections.singletonList(new AliasMetadata.Builder("d").build())) + .build(); + final String[] concreteIndices = {"c"}; + assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); + ImmutableOpenMap> result = + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state); + assertThat(result.size(), equalTo(1)); + assertThat(result.get("c").size(), equalTo(1)); + } + + public ClusterState systemIndexTestClusterState() { + return ClusterState.builder(ClusterState.EMPTY_STATE) + .metadata(Metadata.builder() + .put(IndexMetadata.builder("a").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0)) + .put(IndexMetadata.builder(".b").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0) + .system(true).putAlias(AliasMetadata.builder(".y"))) + .put(IndexMetadata.builder("c").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0) + .putAlias(AliasMetadata.builder("d"))) + .build()) + .build(); + } + + } From 51e4d50bd686d3b7891d334270d85011757f036e Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 30 Sep 2020 14:19:39 -0600 Subject: [PATCH 123/161] Specify index in "Test wildcard job id" so we don't get a warning --- .../test/resources/rest-api-spec/test/ml/custom_all_field.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml index f6bf53fb289db..eefd9b937cbec 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/custom_all_field.yml @@ -148,6 +148,7 @@ setup: - do: search: + index: .ml-anomalies-shared expand_wildcards: all rest_total_hits_as_int: true body: { query: { bool: { must: [ From 661ce27597074b01737cc90965787abbd4802eec Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 30 Sep 2020 14:20:11 -0600 Subject: [PATCH 124/161] Accept warning in SQL test that might trigger it --- .../sql/qa/security/RestSqlSecurityIT.java | 65 ++++++++++++------- 1 file changed, 41 insertions(+), 24 deletions(-) diff --git a/x-pack/plugin/sql/qa/server/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/RestSqlSecurityIT.java b/x-pack/plugin/sql/qa/server/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/RestSqlSecurityIT.java index abb12a3212241..41bcbb74722cd 100644 --- a/x-pack/plugin/sql/qa/server/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/RestSqlSecurityIT.java +++ b/x-pack/plugin/sql/qa/server/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/RestSqlSecurityIT.java @@ -60,13 +60,13 @@ public void queryWorksAsAdmin() throws Exception { ); expected.put("rows", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); - assertResponse(expected, runSql(null, mode, "SELECT * FROM test ORDER BY a")); + assertResponse(expected, runSql(null, mode, "SELECT * FROM test ORDER BY a", false)); } @Override public void expectMatchesAdmin(String adminSql, String user, String userSql) throws Exception { String mode = randomMode(); - assertResponse(runSql(null, mode, adminSql), runSql(user, mode, userSql)); + assertResponse(runSql(null, mode, adminSql, false), runSql(user, mode, userSql, false)); } @Override @@ -75,13 +75,13 @@ public void expectScrollMatchesAdmin(String adminSql, String user, String userSq Map adminResponse = runSql( null, new StringEntity(query(adminSql).mode(mode).fetchSize(1).toString(), ContentType.APPLICATION_JSON), - mode - ); + mode, + false); Map otherResponse = runSql( user, new StringEntity(query(adminSql).mode(mode).fetchSize(1).toString(), ContentType.APPLICATION_JSON), - mode - ); + mode, + false); String adminCursor = (String) adminResponse.remove("cursor"); String otherCursor = (String) otherResponse.remove("cursor"); @@ -92,13 +92,13 @@ public void expectScrollMatchesAdmin(String adminSql, String user, String userSq adminResponse = runSql( null, new StringEntity(cursor(adminCursor).mode(mode).toString(), ContentType.APPLICATION_JSON), - mode - ); + mode, + false); otherResponse = runSql( user, new StringEntity(cursor(otherCursor).mode(mode).toString(), ContentType.APPLICATION_JSON), - mode - ); + mode, + false); adminCursor = (String) adminResponse.remove("cursor"); otherCursor = (String) otherResponse.remove("cursor"); assertResponse(adminResponse, otherResponse); @@ -131,7 +131,7 @@ public void expectDescribe(Map> columns, String user) throw } expected.put("rows", rows); - assertResponse(expected, runSql(user, mode, "DESCRIBE test")); + assertResponse(expected, runSql(user, mode, "DESCRIBE test", false)); } @Override @@ -153,7 +153,8 @@ public void expectShowTables(List tables, String user) throws Exception } expected.put("rows", rows); - Map actual = runSql(user, mode, "SHOW TABLES"); + // Allow system index deprecation warnings, because this may return `.security*` indices. + Map actual = runSql(user, mode, "SHOW TABLES", true); /* * Security automatically creates either a `.security` or a * `.security6` index but it might not have created the index @@ -169,21 +170,21 @@ public void expectShowTables(List tables, String user) throws Exception @Override public void expectForbidden(String user, String sql) { - ResponseException e = expectThrows(ResponseException.class, () -> runSql(user, randomMode(), sql)); + ResponseException e = expectThrows(ResponseException.class, () -> runSql(user, randomMode(), sql, false)); assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(403)); assertThat(e.getMessage(), containsString("unauthorized")); } @Override public void expectUnknownIndex(String user, String sql) { - ResponseException e = expectThrows(ResponseException.class, () -> runSql(user, randomMode(), sql)); + ResponseException e = expectThrows(ResponseException.class, () -> runSql(user, randomMode(), sql, false)); assertThat(e.getResponse().getStatusLine().getStatusCode(), equalTo(400)); assertThat(e.getMessage(), containsString("Unknown index")); } @Override public void expectUnknownColumn(String user, String sql, String column) throws Exception { - ResponseException e = expectThrows(ResponseException.class, () -> runSql(user, randomMode(), sql)); + ResponseException e = expectThrows(ResponseException.class, () -> runSql(user, randomMode(), sql, false)); assertThat(e.getMessage(), containsString("Unknown column [" + column + "]")); } @@ -195,17 +196,33 @@ public void checkNoMonitorMain(String user) throws Exception { expectMatchesAdmin("DESCRIBE test", user, "DESCRIBE test"); } - private static Map runSql(@Nullable String asUser, String mode, String sql) throws IOException { - return runSql(asUser, new StringEntity(query(sql).mode(mode).toString(), ContentType.APPLICATION_JSON), mode); + private static Map runSql(@Nullable String asUser, String mode, String sql, + boolean allowSystemIndexDeprecationWarning) throws IOException { + return runSql(asUser, new StringEntity(query(sql).mode(mode).toString(), ContentType.APPLICATION_JSON), mode, + allowSystemIndexDeprecationWarning); } - private static Map runSql(@Nullable String asUser, HttpEntity entity, String mode) throws IOException { + private static Map runSql(@Nullable String asUser, HttpEntity entity, String mode, + boolean allowSystemIndexDeprecationWarning) throws IOException { Request request = new Request("POST", SQL_QUERY_REST_ENDPOINT); + RequestOptions.Builder options = request.getOptions().toBuilder(); if (asUser != null) { - RequestOptions.Builder options = request.getOptions().toBuilder(); options.addHeader("es-security-runas-user", asUser); - request.setOptions(options); } + if (allowSystemIndexDeprecationWarning) { + options.setWarningsHandler(warnings -> { + if (warnings.isEmpty()) { + // No warnings is OK + return false; + } else if (warnings.size() > 1) { + return true; + } else { + String warning = warnings.get(0); + return warning.startsWith("this request accesses system indices: ") == false; + } + }); + } + request.setOptions(options); request.setEntity(entity); return toMap(client().performRequest(request), mode); } @@ -251,8 +268,8 @@ public void testHijackScrollFails() throws Exception { Map adminResponse = RestActions.runSql( null, new StringEntity(query("SELECT * FROM test").mode(mode).fetchSize(1).toString(), ContentType.APPLICATION_JSON), - mode - ); + mode, + false); String cursor = (String) adminResponse.remove("cursor"); assertNotNull(cursor); @@ -262,8 +279,8 @@ public void testHijackScrollFails() throws Exception { () -> RestActions.runSql( "full_access", new StringEntity(cursor(cursor).mode(mode).toString(), ContentType.APPLICATION_JSON), - mode - ) + mode, + false) ); // TODO return a better error message for bad scrolls assertThat(e.getMessage(), containsString("No search context found for id")); From 5c07feab739165e8e232d4cfa718309f1cfe2855 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 30 Sep 2020 14:39:39 -0600 Subject: [PATCH 125/161] Allow system index access in APIs we're confident we want it --- .../cluster/RestClusterAllocationExplainAction.java | 5 +++++ .../action/admin/cluster/RestClusterRerouteAction.java | 5 +++++ .../action/admin/cluster/RestClusterStateAction.java | 5 +++++ .../admin/indices/RestIndicesShardStoresAction.java | 5 +++++ .../action/admin/indices/RestIndicesStatsAction.java | 5 +++++ .../rest/action/admin/indices/RestRecoveryAction.java | 5 +++++ .../elasticsearch/rest/action/cat/RestHealthAction.java | 6 ++++++ .../elasticsearch/rest/action/cat/RestIndicesAction.java | 5 +++++ .../rest/action/cat/RestSegmentsAction.java | 5 +++++ .../elasticsearch/rest/action/cat/RestShardsAction.java | 9 +++++++-- 10 files changed, 53 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterAllocationExplainAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterAllocationExplainAction.java index a156270d0d7ce..55ce587e3d8f5 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterAllocationExplainAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterAllocationExplainAction.java @@ -55,6 +55,11 @@ public String getName() { return "cluster_allocation_explain_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { ClusterAllocationExplainRequest req; diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterRerouteAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterRerouteAction.java index e5b749c56ad40..50f63d651ba14 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterRerouteAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterRerouteAction.java @@ -70,6 +70,11 @@ public String getName() { return "cluster_reroute_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { ClusterRerouteRequest clusterRerouteRequest = createRequest(request); diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java index de74532ee6d0c..2b592874f07d1 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestClusterStateAction.java @@ -69,6 +69,11 @@ public List routes() { new Route(GET, "/_cluster/state/{metric}/{indices}")); } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { final ClusterStateRequest clusterStateRequest = Requests.clusterStateRequest(); diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesShardStoresAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesShardStoresAction.java index b26299f19767b..0c7b897d1c09b 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesShardStoresAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesShardStoresAction.java @@ -55,6 +55,11 @@ public String getName() { return "indices_shard_stores_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { IndicesShardStoresRequest indicesShardStoresRequest = new IndicesShardStoresRequest( diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java index 2f63b7b5b47ab..67a88f2cd246f 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestIndicesStatsAction.java @@ -57,6 +57,11 @@ public String getName() { return "indices_stats_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + static final Map> METRICS; static { diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRecoveryAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRecoveryAction.java index 5f8a9424773e4..1bd1348e497c7 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRecoveryAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRecoveryAction.java @@ -49,6 +49,11 @@ public String getName() { return "recovery_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestHealthAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestHealthAction.java index 45ea2a90ef4e4..d2aca04bbdbf4 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestHealthAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestHealthAction.java @@ -44,11 +44,17 @@ public String getName() { return "cat_health_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override protected void documentation(StringBuilder sb) { sb.append("/_cat/health\n"); } + @Override public RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) { ClusterHealthRequest clusterHealthRequest = new ClusterHealthRequest(); 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 d313cb900f613..5637918353dda 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 @@ -81,6 +81,11 @@ public String getName() { return "cat_indices_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override protected void documentation(StringBuilder sb) { sb.append("/_cat/indices\n"); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestSegmentsAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestSegmentsAction.java index c3f5c3c9d0ff3..08be5c7a7f957 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestSegmentsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestSegmentsAction.java @@ -55,6 +55,11 @@ public String getName() { return "cat_segments_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override protected RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) { final String[] indices = Strings.splitStringByCommaToArray(request.param("index")); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java index 959aeac6ff0ac..ef6b6cf60c639 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java @@ -72,6 +72,11 @@ public String getName() { return "cat_shards_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override protected void documentation(StringBuilder sb) { sb.append("/_cat/shards\n"); @@ -203,7 +208,7 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("path.data", "alias:pd,dataPath;default:false;text-align:right;desc:shard data path"); table.addCell("path.state", "alias:ps,statsPath;default:false;text-align:right;desc:shard state path"); - + table.addCell("bulk.total_operations", "alias:bto,bulkTotalOperations;default:false;text-align:right;desc:number of bulk shard ops"); table.addCell("bulk.total_time", "alias:btti,bulkTotalTime;default:false;text-align:right;desc:time spend in shard bulk"); @@ -367,7 +372,7 @@ Table buildTable(RestRequest request, ClusterStateResponse state, IndicesStatsRe table.addCell(getOrNull(shardStats, ShardStats::getDataPath, s -> s)); table.addCell(getOrNull(shardStats, ShardStats::getStatePath, s -> s)); - + table.addCell(getOrNull(commonStats, CommonStats::getBulk, BulkStats::getTotalOperations)); table.addCell(getOrNull(commonStats, CommonStats::getBulk, BulkStats::getTotalTime)); table.addCell(getOrNull(commonStats, CommonStats::getBulk, BulkStats::getTotalSizeInBytes)); From 70136919984f45a98a4f6e9c6d2da0dcb0c9f273 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Wed, 30 Sep 2020 16:50:30 -0600 Subject: [PATCH 126/161] Spotless --- .../sql/qa/security/RestSqlSecurityIT.java | 42 +++++++++++++------ 1 file changed, 30 insertions(+), 12 deletions(-) diff --git a/x-pack/plugin/sql/qa/server/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/RestSqlSecurityIT.java b/x-pack/plugin/sql/qa/server/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/RestSqlSecurityIT.java index 41bcbb74722cd..e46dc26a3dc5d 100644 --- a/x-pack/plugin/sql/qa/server/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/RestSqlSecurityIT.java +++ b/x-pack/plugin/sql/qa/server/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/RestSqlSecurityIT.java @@ -76,12 +76,14 @@ public void expectScrollMatchesAdmin(String adminSql, String user, String userSq null, new StringEntity(query(adminSql).mode(mode).fetchSize(1).toString(), ContentType.APPLICATION_JSON), mode, - false); + false + ); Map otherResponse = runSql( user, new StringEntity(query(adminSql).mode(mode).fetchSize(1).toString(), ContentType.APPLICATION_JSON), mode, - false); + false + ); String adminCursor = (String) adminResponse.remove("cursor"); String otherCursor = (String) otherResponse.remove("cursor"); @@ -93,12 +95,14 @@ public void expectScrollMatchesAdmin(String adminSql, String user, String userSq null, new StringEntity(cursor(adminCursor).mode(mode).toString(), ContentType.APPLICATION_JSON), mode, - false); + false + ); otherResponse = runSql( user, new StringEntity(cursor(otherCursor).mode(mode).toString(), ContentType.APPLICATION_JSON), mode, - false); + false + ); adminCursor = (String) adminResponse.remove("cursor"); otherCursor = (String) otherResponse.remove("cursor"); assertResponse(adminResponse, otherResponse); @@ -196,14 +200,26 @@ public void checkNoMonitorMain(String user) throws Exception { expectMatchesAdmin("DESCRIBE test", user, "DESCRIBE test"); } - private static Map runSql(@Nullable String asUser, String mode, String sql, - boolean allowSystemIndexDeprecationWarning) throws IOException { - return runSql(asUser, new StringEntity(query(sql).mode(mode).toString(), ContentType.APPLICATION_JSON), mode, - allowSystemIndexDeprecationWarning); + private static Map runSql( + @Nullable String asUser, + String mode, + String sql, + boolean allowSystemIndexDeprecationWarning + ) throws IOException { + return runSql( + asUser, + new StringEntity(query(sql).mode(mode).toString(), ContentType.APPLICATION_JSON), + mode, + allowSystemIndexDeprecationWarning + ); } - private static Map runSql(@Nullable String asUser, HttpEntity entity, String mode, - boolean allowSystemIndexDeprecationWarning) throws IOException { + private static Map runSql( + @Nullable String asUser, + HttpEntity entity, + String mode, + boolean allowSystemIndexDeprecationWarning + ) throws IOException { Request request = new Request("POST", SQL_QUERY_REST_ENDPOINT); RequestOptions.Builder options = request.getOptions().toBuilder(); if (asUser != null) { @@ -269,7 +285,8 @@ public void testHijackScrollFails() throws Exception { null, new StringEntity(query("SELECT * FROM test").mode(mode).fetchSize(1).toString(), ContentType.APPLICATION_JSON), mode, - false); + false + ); String cursor = (String) adminResponse.remove("cursor"); assertNotNull(cursor); @@ -280,7 +297,8 @@ public void testHijackScrollFails() throws Exception { "full_access", new StringEntity(cursor(cursor).mode(mode).toString(), ContentType.APPLICATION_JSON), mode, - false) + false + ) ); // TODO return a better error message for bad scrolls assertThat(e.getMessage(), containsString("No search context found for id")); From 2e138fd37307a7071614d241b06978f5eb30bf88 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 1 Oct 2020 11:17:49 -0600 Subject: [PATCH 127/161] Respect system index access header when re-adding deprecation in Get Alias API --- .../alias/get/TransportGetAliasesAction.java | 9 ++++-- .../metadata/IndexNameExpressionResolver.java | 2 +- .../get/TransportGetAliasesActionTests.java | 29 ++++++++++++++----- 3 files changed, 29 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java index c0e867a939c4c..925faf3e02083 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java @@ -81,7 +81,8 @@ protected void masterOperation(Task task, GetAliasesRequest request, ClusterStat concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); } ImmutableOpenMap> aliases = state.metadata().findAliases(request, concreteIndices); - listener.onResponse(new GetAliasesResponse(postProcess(request, concreteIndices, aliases, state))); + listener.onResponse(new GetAliasesResponse(postProcess(request, concreteIndices, aliases, state, + indexNameExpressionResolver.isSystemIndexAccessAllowed()))); } /** @@ -89,7 +90,7 @@ protected void masterOperation(Task task, GetAliasesRequest request, ClusterStat */ static ImmutableOpenMap> postProcess(GetAliasesRequest request, String[] concreteIndices, ImmutableOpenMap> aliases, - ClusterState state) { + ClusterState state, boolean systemIndexAccessAllowed) { boolean noAliasesSpecified = request.getOriginalAliases() == null || request.getOriginalAliases().length == 0; ImmutableOpenMap.Builder> mapBuilder = ImmutableOpenMap.builder(aliases); for (String index : concreteIndices) { @@ -99,7 +100,9 @@ static ImmutableOpenMap> postProcess(GetAliasesReque } } final ImmutableOpenMap> finalResponse = mapBuilder.build(); - checkSystemIndexAccess(state, finalResponse); + if (systemIndexAccessAllowed == false) { + checkSystemIndexAccess(state, finalResponse); + } return finalResponse; } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 124abfc398c55..46139c3fa3cd1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -712,7 +712,7 @@ boolean isPatternMatchingAllIndices(Metadata metadata, String[] indicesOrAliases return false; } - private boolean isSystemIndexAccessAllowed() { + public boolean isSystemIndexAccessAllowed() { return Booleans.parseBoolean(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY), true); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java index 9659fd4523c82..8960f663ccb5d 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java @@ -39,7 +39,7 @@ public void testPostProcess() { .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) .build(); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE); + TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE, false); assertThat(result.size(), equalTo(3)); assertThat(result.get("a").size(), equalTo(0)); assertThat(result.get("b").size(), equalTo(1)); @@ -50,7 +50,7 @@ public void testPostProcess() { aliases = ImmutableOpenMap.>builder() .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) .build(); - result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE); + result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE, false); assertThat(result.size(), equalTo(3)); assertThat(result.get("a").size(), equalTo(0)); assertThat(result.get("b").size(), equalTo(1)); @@ -60,7 +60,7 @@ public void testPostProcess() { aliases = ImmutableOpenMap.>builder() .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) .build(); - result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE); + result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE, false); assertThat(result.size(), equalTo(1)); assertThat(result.get("b").size(), equalTo(1)); } @@ -76,7 +76,7 @@ public void testDeprecationWarningEmittedForTotalWildcard() { final String[] concreteIndices = {"a", ".b", "c"}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state); + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false); assertThat(result.size(), equalTo(3)); assertThat(result.get("a").size(), equalTo(0)); assertThat(result.get(".b").size(), equalTo(1)); @@ -96,7 +96,7 @@ public void testDeprecationWarningEmittedWhenSystemIndexIsRequested() { final String[] concreteIndices = {".b"}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state); + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false); assertThat(result.size(), equalTo(1)); assertThat(result.get(".b").size(), equalTo(1)); assertWarnings("this request accesses system indices: [.b], but in a future major version, direct access to system " + @@ -113,13 +113,28 @@ public void testDeprecationWarningEmittedWhenSystemIndexIsRequestedByAlias() { final String[] concreteIndices = {"a", ".b", "c"}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state); + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false); assertThat(result.size(), equalTo(1)); assertThat(result.get(".b").size(), equalTo(1)); assertWarnings("this request accesses system indices: [.b], but in a future major version, direct access to system " + "indices will be prevented by default"); } + public void testDeprecationWarningNotEmittedWhenSystemAccessAllowed() { + ClusterState state = systemIndexTestClusterState(); + + GetAliasesRequest request = new GetAliasesRequest(".y"); + ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() + .fPut(".b", Collections.singletonList(new AliasMetadata.Builder(".y").build())) + .build(); + final String[] concreteIndices = {"a", ".b", "c"}; + assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); + ImmutableOpenMap> result = + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, true); + assertThat(result.size(), equalTo(1)); + assertThat(result.get(".b").size(), equalTo(1)); + } + /** * Ensures that deprecation warnings are not emitted when */ @@ -134,7 +149,7 @@ public void testDeprecationWarningNotEmittedWhenOnlyNonsystemIndexRequested() { final String[] concreteIndices = {"c"}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state); + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false); assertThat(result.size(), equalTo(1)); assertThat(result.get("c").size(), equalTo(1)); } From df094c3e0504b05a3b6526ad9e9383c4d4b1882e Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 1 Oct 2020 11:26:29 -0600 Subject: [PATCH 128/161] Handle Get Alias deprecation warning in upgrade test --- .../org/elasticsearch/upgrades/FullClusterRestartIT.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index 74bb27d5a3603..17cc250814327 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -1391,6 +1391,8 @@ public void testResize() throws Exception { @SuppressWarnings("unchecked") public void testSystemIndexMetadataIsUpgraded() throws Exception { + final String systemIndexWarning = "this request accesses system indices: [.tasks], but in a future major version, direct " + + "access to system indices will be prevented by default"; if (isRunningAgainstOldCluster()) { // create index Request createTestIndex = new Request("PUT", "/test_index_old"); @@ -1428,8 +1430,6 @@ public void testSystemIndexMetadataIsUpgraded() throws Exception { getTasksIndex.addParameter("allow_no_indices", "false"); getTasksIndex.setOptions(expectVersionSpecificWarnings(v -> { - final String systemIndexWarning = "this request accesses system indices: [.tasks], but in a future major version, direct " + - "access to system indices will be prevented by default"; v.current(systemIndexWarning); v.compatible(systemIndexWarning); })); @@ -1475,6 +1475,10 @@ public void testSystemIndexMetadataIsUpgraded() throws Exception { if (tasksCreatedVersion.before(SYSTEM_INDEX_ENFORCEMENT_VERSION)) { // Verify that the alias survived the upgrade Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); + getAliasRequest.setOptions(expectVersionSpecificWarnings(v -> { + v.current(systemIndexWarning); + v.compatible(systemIndexWarning); + })); Map aliasResponse = entityAsMap(client().performRequest(getAliasRequest)); assertThat(aliasResponse, hasKey(".tasks")); assertThat(aliasResponse, hasKey("test_index_reindex")); From 2cb62db7ace2b30a47387b91d2b181e3e9e5cd6a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 1 Oct 2020 13:28:59 -0600 Subject: [PATCH 129/161] Allow data streams stats action to only resolve to data streams --- .../node/TransportBroadcastByNodeAction.java | 13 ++++++++- .../metadata/IndexNameExpressionResolver.java | 3 +- .../DataStreamsStatsTransportAction.java | 29 +++++++++++-------- 3 files changed, 31 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeAction.java b/server/src/main/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeAction.java index b5fc85d6234e5..d914d6ad3037a 100644 --- a/server/src/main/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeAction.java @@ -220,6 +220,17 @@ protected abstract Response newResponse(Request request, int totalShards, int su */ protected abstract ClusterBlockException checkRequestBlock(ClusterState state, Request request, String[] concreteIndices); + /** + * Resolves a list of concrete index names. Override this if index names should be resolved differently than normal. + * + * @param clusterState the cluster state + * @param request the underlying request + * @return a list of concrete index names that this action should operate on + */ + protected String[] resolveConcreteIndexNames(ClusterState clusterState, Request request) { + return indexNameExpressionResolver.concreteIndexNames(clusterState, request); + } + @Override protected void doExecute(Task task, Request request, ActionListener listener) { new AsyncAction(task, request, listener).start(); @@ -249,7 +260,7 @@ protected AsyncAction(Task task, Request request, ActionListener liste throw globalBlockException; } - String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(clusterState, request); + String[] concreteIndices = resolveConcreteIndexNames(clusterState, request); ClusterBlockException requestBlockException = checkRequestBlock(clusterState, request, concreteIndices); if (requestBlockException != null) { throw requestBlockException; diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 46139c3fa3cd1..b7d0632024ba5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -139,7 +139,8 @@ public String[] concreteIndexNames(ClusterState state, IndicesOptions options, I } public List dataStreamNames(ClusterState state, IndicesOptions options, String... indexExpressions) { - Context context = new Context(state, options, false, false, true, true, isSystemIndexAccessAllowed()); + // Allow system index access - they'll be filtered out below as there's no such thing (yet) as system data streams + Context context = new Context(state, options, false, false, true, true, true); if (indexExpressions == null || indexExpressions.length == 0) { indexExpressions = new String[]{"*"}; } diff --git a/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/action/DataStreamsStatsTransportAction.java b/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/action/DataStreamsStatsTransportAction.java index bccac4e2ed711..18be5fce55ccd 100644 --- a/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/action/DataStreamsStatsTransportAction.java +++ b/x-pack/plugin/data-streams/src/main/java/org/elasticsearch/xpack/datastreams/action/DataStreamsStatsTransportAction.java @@ -95,17 +95,13 @@ protected ClusterBlockException checkRequestBlock( return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, concreteIndices); } - private List dataStreamNames(ClusterState clusterState, DataStreamsStatsAction.Request request) { - String[] requestIndices = request.indices(); - if (requestIndices == null || requestIndices.length == 0) { - requestIndices = new String[] { "*" }; - } - return indexNameExpressionResolver.dataStreamNames(clusterState, request.indicesOptions(), requestIndices); - } - @Override - protected ShardsIterator shards(ClusterState clusterState, DataStreamsStatsAction.Request request, String[] concreteIndices) { - List abstractionNames = dataStreamNames(clusterState, request); + protected String[] resolveConcreteIndexNames(ClusterState clusterState, DataStreamsStatsAction.Request request) { + List abstractionNames = indexNameExpressionResolver.dataStreamNames( + clusterState, + request.indicesOptions(), + request.indices() + ); SortedMap indicesLookup = clusterState.getMetadata().getIndicesLookup(); String[] concreteDatastreamIndices = abstractionNames.stream().flatMap(abstractionName -> { @@ -119,7 +115,12 @@ protected ShardsIterator shards(ClusterState clusterState, DataStreamsStatsActio return Stream.empty(); } }).toArray(String[]::new); - return clusterState.getRoutingTable().allShards(concreteDatastreamIndices); + return concreteDatastreamIndices; + } + + @Override + protected ShardsIterator shards(ClusterState clusterState, DataStreamsStatsAction.Request request, String[] concreteIndices) { + return clusterState.getRoutingTable().allShards(concreteIndices); } @Override @@ -171,7 +172,11 @@ protected DataStreamsStatsAction.Response newResponse( // Collect the number of backing indices from the cluster state. If every shard operation for an index fails, // or if a backing index simply has no shards allocated, it would be excluded from the counts if we only used // shard results to calculate. - List abstractionNames = dataStreamNames(clusterState, request); + List abstractionNames = indexNameExpressionResolver.dataStreamNames( + clusterState, + request.indicesOptions(), + request.indices() + ); for (String abstractionName : abstractionNames) { IndexAbstraction indexAbstraction = indicesLookup.get(abstractionName); assert indexAbstraction != null; From a00a0c79053e6d4962e1447a6c1e50837e014bb7 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 1 Oct 2020 14:01:28 -0600 Subject: [PATCH 130/161] Allow deprecation warning on another alias call --- .../elasticsearch/upgrades/SystemIndicesUpgradeIT.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java index 6c858f01aca6c..65f9c66bff549 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SystemIndicesUpgradeIT.java @@ -35,6 +35,8 @@ public class SystemIndicesUpgradeIT extends AbstractRollingTestCase { @SuppressWarnings("unchecked") public void testSystemIndicesUpgrades() throws Exception { + final String systemIndexWarning = "this request accesses system indices: [.tasks], but in a future major version, direct " + + "access to system indices will be prevented by default"; if (CLUSTER_TYPE == ClusterType.OLD) { // create index Request createTestIndex = new Request("PUT", "/test_index_old"); @@ -72,8 +74,6 @@ public void testSystemIndicesUpgrades() throws Exception { getTasksIndex.addParameter("allow_no_indices", "false"); getTasksIndex.setOptions(expectVersionSpecificWarnings(v -> { - final String systemIndexWarning = "this request accesses system indices: [.tasks], but in a future major version, direct " + - "access to system indices will be prevented by default"; v.current(systemIndexWarning); v.compatible(systemIndexWarning); })); @@ -119,6 +119,10 @@ public void testSystemIndicesUpgrades() throws Exception { if (tasksCreatedVersion.before(SYSTEM_INDEX_ENFORCEMENT_VERSION)) { // Verify that the alias survived the upgrade Request getAliasRequest = new Request("GET", "/_alias/test-system-alias"); + getAliasRequest.setOptions(expectVersionSpecificWarnings(v -> { + v.current(systemIndexWarning); + v.compatible(systemIndexWarning); + })); Map aliasResponse = entityAsMap(client().performRequest(getAliasRequest)); assertThat(aliasResponse, hasKey(".tasks")); assertThat(aliasResponse, hasKey("test_index_reindex")); From aaa9759531de48acdf323991048fe8e79b992a7d Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 1 Oct 2020 15:08:37 -0600 Subject: [PATCH 131/161] Allow _cat/aliases to access system indices by default because it's weird otherwise --- .../org/elasticsearch/rest/action/cat/RestAliasAction.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestAliasAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestAliasAction.java index 451d67d246017..81aa0d6c55dd8 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestAliasAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestAliasAction.java @@ -48,6 +48,11 @@ public String getName() { return "cat_alias_action"; } + @Override + public boolean allowSystemIndexAccessByDefault() { + return true; + } + @Override protected RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) { final GetAliasesRequest getAliasesRequest = request.hasParam("alias") ? From 1f989c766977e0eda0f51da334f096d848ee867c Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 1 Oct 2020 15:09:08 -0600 Subject: [PATCH 132/161] Allow deprecation warning on wildcard Get Aliases call in MlJobIT --- .../xpack/ml/integration/MlJobIT.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/MlJobIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/MlJobIT.java index 7f6892842ea89..721a566cb18c9 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/MlJobIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/MlJobIT.java @@ -7,6 +7,7 @@ import org.apache.http.util.EntityUtils; import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.common.settings.Settings; @@ -34,7 +35,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasEntry; @@ -818,7 +818,19 @@ public void testDelete_multipleRequest() throws Exception { } private String getAliases() throws IOException { - Response response = client().performRequest(new Request("GET", "/_aliases")); + final Request aliasesRequest = new Request("GET", "/_aliases"); + // Allow system index deprecation warnings - this can be removed once system indices are omitted from responses rather than + // triggering a deprecation warning. + aliasesRequest.setOptions(RequestOptions.DEFAULT.toBuilder().setWarningsHandler(warnings -> { + if (warnings.isEmpty()) { + return false; + } else if (warnings.size() > 1) { + return true; + } else { + return warnings.get(0).startsWith("this request accesses system indices:") == false; + } + }).build()); + Response response = client().performRequest(aliasesRequest); return EntityUtils.toString(response.getEntity()); } From cae36a5ff97ff98510939b1b58401c2572e275bf Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Thu, 1 Oct 2020 16:33:49 -0600 Subject: [PATCH 133/161] Remove unnecessary semicolon --- .../cluster/metadata/IndexNameExpressionResolver.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index b7d0632024ba5..34f67ce6a5f25 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -339,7 +339,6 @@ private void checkSystemIndexAccess(Context context, Metadata metadata, Set Date: Fri, 2 Oct 2020 15:24:01 -0600 Subject: [PATCH 134/161] Fix compilation after merge --- .../action/TransportStartDataFrameAnalyticsActionTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java index 9322e46aaf980..7183550277a3f 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsActionTests.java @@ -23,8 +23,8 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; @@ -231,7 +231,7 @@ private static TaskExecutor createTaskExecutor() { mock(DataFrameAnalyticsManager.class), mock(DataFrameAnalyticsAuditor.class), mock(MlMemoryTracker.class), - new IndexNameExpressionResolver(), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), mock(IndexTemplateConfig.class)); } From 0e302e6c75e75684b2d0f5b3b36bcbcd5836cdc7 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 2 Oct 2020 16:56:12 -0600 Subject: [PATCH 135/161] Remove "feature flag" for deprecation warnings --- .../elasticsearch/http/SystemIndexRestIT.java | 4 --- .../metadata/IndexNameExpressionResolver.java | 32 ++--------------- .../elasticsearch/indices/SystemIndices.java | 35 ------------------- .../IndexNameExpressionResolverTests.java | 10 ------ 4 files changed, 3 insertions(+), 78 deletions(-) diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java index 70934124b156e..12bbd12485853 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SystemIndexRestIT.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.indices.SystemIndexDescriptor; -import org.elasticsearch.indices.SystemIndices; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.SystemIndexPlugin; import org.elasticsearch.rest.BaseRestHandler; @@ -49,7 +48,6 @@ import java.util.Map; import java.util.function.Supplier; -import static org.elasticsearch.indices.SystemIndices.AccessBehavior.DEPRECATED; import static org.elasticsearch.test.rest.ESRestTestCase.entityAsMap; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasKey; @@ -65,8 +63,6 @@ protected Collection> nodePlugins() { } public void testSystemIndexAccessBlockedByDefault() throws Exception { - assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", - SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); // create index { Request putDocRequest = new Request("POST", "/_sys_index_test/add_doc/42"); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 34f67ce6a5f25..7d078cddd51de 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -41,7 +41,6 @@ import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.indices.IndexClosedException; import org.elasticsearch.indices.InvalidIndexNameException; -import org.elasticsearch.indices.SystemIndices; import java.time.Instant; import java.time.ZoneId; @@ -63,8 +62,6 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; -import static org.elasticsearch.indices.SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR; - public class IndexNameExpressionResolver { public static DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(IndexNameExpressionResolver.class); @@ -324,30 +321,14 @@ private void checkSystemIndexAccess(Context context, Metadata metadata, Set innerResolve(Context context, List expressions, Indi } else if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM && context.includeDataStreams() == false) { throw indexNotFoundException(expression); - } else if (SYSTEM_INDEX_ACCESS_BEHAVIOR == SystemIndices.AccessBehavior.RESTRICTED - && context.isSystemIndexAccessAllowed() == false && indexAbstraction.isSystem()) { - throw indexNotFoundException(expression); } } if (add) { @@ -1034,10 +1012,6 @@ private static Set expand(Context context, IndexMetadata.State excludeSt for (Map.Entry entry : matches.entrySet()) { String aliasOrIndexName = entry.getKey(); IndexAbstraction indexAbstraction = entry.getValue(); - if (SYSTEM_INDEX_ACCESS_BEHAVIOR == SystemIndices.AccessBehavior.RESTRICTED - && context.isSystemIndexAccessAllowed() == false && indexAbstraction.isSystem()) { - continue; - } if (indexAbstraction.isHidden() == false || includeHidden || implicitHiddenMatch(aliasOrIndexName, expression)) { if (context.isPreserveAliases() && indexAbstraction.getType() == IndexAbstraction.Type.ALIAS) { diff --git a/server/src/main/java/org/elasticsearch/indices/SystemIndices.java b/server/src/main/java/org/elasticsearch/indices/SystemIndices.java index a17d3357b8627..ae4a64111a588 100644 --- a/server/src/main/java/org/elasticsearch/indices/SystemIndices.java +++ b/server/src/main/java/org/elasticsearch/indices/SystemIndices.java @@ -24,7 +24,6 @@ import org.apache.lucene.util.automaton.CharacterRunAutomaton; import org.apache.lucene.util.automaton.MinimizationOperations; import org.apache.lucene.util.automaton.Operations; -import org.elasticsearch.Build; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.regex.Regex; @@ -35,7 +34,6 @@ import java.util.Comparator; import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -49,23 +47,6 @@ * to reduce the locations within the code that need to deal with {@link SystemIndexDescriptor}s. */ public class SystemIndices { - public static final AccessBehavior SYSTEM_INDEX_ACCESS_BEHAVIOR; - - static { - final String property = System.getProperty("es.system_index_access_behavior"); - if (property == null) { - SYSTEM_INDEX_ACCESS_BEHAVIOR = Build.CURRENT.isSnapshot() ? AccessBehavior.DEPRECATED : AccessBehavior.ALLOWED; - } else { - try { - SYSTEM_INDEX_ACCESS_BEHAVIOR = AccessBehavior.fromString(property); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException( - "expected es.system_index_access_behavior to be unset or [allowed|deprecated|restricted] but was [" + property + "]", e - ); - } - } - } - private static final Map> SERVER_SYSTEM_INDEX_DESCRIPTORS = Map.of( TaskResultsService.class.getName(), List.of(new SystemIndexDescriptor(TASK_INDEX + "*", "Task Result Index")) ); @@ -190,20 +171,4 @@ private static Map> buildSystemIndexDe }); return Map.copyOf(map); } - - public enum AccessBehavior { - ALLOWED, - DEPRECATED, - RESTRICTED; - - public static AccessBehavior fromString(String value) { - return AccessBehavior.valueOf(value.toUpperCase(Locale.ROOT)); - } - - - @Override - public String toString() { - return name().toLowerCase(Locale.ROOT); - } - } } 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 b3120fa1e9b7c..ade851cf15f9d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -40,7 +40,6 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.indices.IndexClosedException; import org.elasticsearch.indices.InvalidIndexNameException; -import org.elasticsearch.indices.SystemIndices; import org.elasticsearch.test.ESTestCase; import java.util.Arrays; @@ -58,7 +57,6 @@ import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_HIDDEN_SETTING; import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY; import static org.elasticsearch.common.util.set.Sets.newHashSet; -import static org.elasticsearch.indices.SystemIndices.AccessBehavior.DEPRECATED; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; import static org.hamcrest.Matchers.arrayWithSize; @@ -1863,8 +1861,6 @@ public void testSingleSystemIndexResolutionAllowed() { } public void testFullWildcardSystemIndexResolutionDeprecated() { - assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", - SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(randomFrom("*", "_all")); @@ -1877,8 +1873,6 @@ public void testFullWildcardSystemIndexResolutionDeprecated() { } public void testSingleSystemIndexResolutionDeprecated() { - assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", - SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".ml-meta"); @@ -1891,8 +1885,6 @@ public void testSingleSystemIndexResolutionDeprecated() { } public void testWildcardSystemIndexReslutionSingleMatchDeprecated() { - assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", - SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".w*"); @@ -1905,8 +1897,6 @@ public void testWildcardSystemIndexReslutionSingleMatchDeprecated() { } public void testWildcardSystemIndexResolutionMultipleMatchesDeprecated() { - assumeTrue("This test assumes access to system indices is deprecated, which is currently only true by default in snapshot builds", - SystemIndices.SYSTEM_INDEX_ACCESS_BEHAVIOR == DEPRECATED); threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); ClusterState state = systemIndexTestClusterState(); SearchRequest request = new SearchRequest(".ml-*"); From 5df4c3fa5d508cb6706977e638f7ca3cef5611b7 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 11:59:50 -0600 Subject: [PATCH 136/161] Refresh no longer allow system index access by default --- .../rest/action/admin/indices/RestRefreshAction.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java index 3a46318c5e4c8..3d9cbd903bb70 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestRefreshAction.java @@ -51,11 +51,6 @@ public String getName() { return "refresh_action"; } - @Override - public boolean allowSystemIndexAccessByDefault() { - return true; - } - @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { RefreshRequest refreshRequest = new RefreshRequest(Strings.splitStringByCommaToArray(request.param("index"))); From 0eb3eb853254b29b4fd712cb4775e68352fe2095 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 12:06:44 -0600 Subject: [PATCH 137/161] Remove unnecessary log line --- .../elasticsearch/xpack/ml/integration/InferenceIngestIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java index ed413dd55815b..5dce3c028d38e 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceIngestIT.java @@ -76,7 +76,6 @@ public void cleanUpData() throws Exception { // There may not be an index to delete, in which case there's no warning return false; } else if (warnings.size() > 1) { - logger.warn("too many warnings: {}", warnings); return true; } // We don't know exactly which indices we're cleaning up in advance, so just accept all system index access warnings. From 591b442e08ed84d43be84b759afe6a67a46dd3fb Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 12:07:19 -0600 Subject: [PATCH 138/161] Don't create a new threadcontext, we already have one --- .../test/java/org/elasticsearch/action/ActionModuleTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java index 861b63a475393..15d2aff8d8787 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java @@ -149,7 +149,7 @@ public String getName() { try { UsageService usageService = new UsageService(); ActionModule actionModule = new ActionModule(settings.getSettings(), - new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), settings.getIndexScopedSettings(), + new IndexNameExpressionResolver(threadPool.getThreadContext()), settings.getIndexScopedSettings(), settings.getClusterSettings(), settings.getSettingsFilter(), threadPool, singletonList(dupsMainAction), null, null, usageService, null); Exception e = expectThrows(IllegalArgumentException.class, () -> actionModule.initRestHandlers(null)); @@ -184,7 +184,7 @@ public List getRestHandlers(Settings settings, RestController restC try { UsageService usageService = new UsageService(); ActionModule actionModule = new ActionModule(settings.getSettings(), - new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), settings.getIndexScopedSettings(), + new IndexNameExpressionResolver(threadPool.getThreadContext()), settings.getIndexScopedSettings(), settings.getClusterSettings(), settings.getSettingsFilter(), threadPool, singletonList(registersFakeHandler), null, null, usageService, null); actionModule.initRestHandlers(null); From 906e024c312bc2f7c5b4862cae6b2cf0b4e45183 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 12:09:12 -0600 Subject: [PATCH 139/161] Don't need to take ThreadContext as a param in dispatchRequest --- .../main/java/org/elasticsearch/rest/RestController.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index 4c7097aa6a93b..0b6fabb54568e 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -219,8 +219,7 @@ public void dispatchBadRequest(final RestChannel channel, final ThreadContext th } } - private void dispatchRequest(RestRequest request, RestChannel channel, RestHandler handler, - ThreadContext threadContext) throws Exception { + private void dispatchRequest(RestRequest request, RestChannel channel, RestHandler handler) throws Exception { final int contentLength = request.contentLength(); if (contentLength > 0) { final XContentType xContentType = request.getXContentType(); @@ -248,7 +247,7 @@ private void dispatchRequest(RestRequest request, RestChannel channel, RestHandl request.ensureSafeBuffers(); } if (handler.allowSystemIndexAccessByDefault() == false) { - threadContext.putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); + client.threadPool().getThreadContext().putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); } handler.handleRequest(request, responseChannel, client); @@ -333,7 +332,7 @@ private void tryAllHandlers(final RestRequest request, final RestChannel channel return; } } else { - dispatchRequest(request, channel, handler, threadContext); + dispatchRequest(request, channel, handler); return; } } From e23a25bf22142b27a6010c42d0190d974f27ee4a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 12:10:09 -0600 Subject: [PATCH 140/161] Javadoc for isSystemIndexAccessAllowed --- .../cluster/metadata/IndexNameExpressionResolver.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index 7d078cddd51de..d5b287c430e27 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -693,6 +693,11 @@ boolean isPatternMatchingAllIndices(Metadata metadata, String[] indicesOrAliases return false; } + /** + * Determines whether or not system index access should be allowed in the current context. + * + * @return True if system index access should be allowed, false otherwise. + */ public boolean isSystemIndexAccessAllowed() { return Booleans.parseBoolean(threadContext.getHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY), true); } From b924de59d287538e1a5595d57c793620cb7c8403 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 12:10:39 -0600 Subject: [PATCH 141/161] Deprecation logger doesn't need to be public --- .../cluster/metadata/IndexNameExpressionResolver.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java index d5b287c430e27..13063d5486b1d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexNameExpressionResolver.java @@ -63,7 +63,7 @@ import java.util.stream.StreamSupport; public class IndexNameExpressionResolver { - public static DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(IndexNameExpressionResolver.class); + private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(IndexNameExpressionResolver.class); public static final String EXCLUDED_DATA_STREAMS_KEY = "es.excluded_ds"; public static final String SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY = "_system_index_access_allowed"; From d3eb24741b36f659875a36af89de8935c2f683b9 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 12:11:10 -0600 Subject: [PATCH 142/161] Doc fix per review --- docs/reference/api-conventions.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/api-conventions.asciidoc b/docs/reference/api-conventions.asciidoc index 6478ffb7a1865..26839eebe0581 100644 --- a/docs/reference/api-conventions.asciidoc +++ b/docs/reference/api-conventions.asciidoc @@ -45,8 +45,8 @@ The defaults settings for the above parameters depend on the API being used. Some indices (hereafter "system indices") are used by various system modules and/or plugins to store state or configuration. These indices are not intended to be accessed directly, and accessing them directly is -deprecated. In the next major version, access to these indices will be prevented -by default to prevent accidental operations that may cause problems with +deprecated. In the next major version, access to these indices will no longer be +allowed to prevent accidental operations that may cause problems with Elasticsearch features which depend on the consistency of data in these indices. From c223029fc074d7ed607cbb5aec456cf08597a616 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 12:17:21 -0600 Subject: [PATCH 143/161] Allow deprecation warning for refreshing all indices --- .../org/elasticsearch/test/rest/ESRestTestCase.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 58e990c61a305..6fcd5bb02af7d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -813,6 +813,16 @@ protected void refreshAllIndices() throws IOException { boolean includeHidden = minimumNodeVersion().onOrAfter(Version.V_7_7_0); Request refreshRequest = new Request("POST", "/_refresh"); refreshRequest.addParameter("expand_wildcards", "open,closed" + (includeHidden ? ",hidden" : "")); + // Allow system index deprecation warnings + refreshRequest.setOptions(RequestOptions.DEFAULT.toBuilder().setWarningsHandler(warnings -> { + if (warnings.isEmpty()) { + return false; + } else if (warnings.size() > 1) { + return true; + } else { + return warnings.get(0).startsWith("this request accesses system indices:") == false; + } + })); client().performRequest(refreshRequest); } From ebabb5ab2ff2db343dea758dcb48e00bf5513085 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 12:55:13 -0600 Subject: [PATCH 144/161] Allow warnings on refresh in reindex YAML tests --- .../rest-api-spec/test/delete_by_query/70_throttle.yml | 3 +++ .../rest-api-spec/test/delete_by_query/80_slices.yml | 4 ++++ .../resources/rest-api-spec/test/reindex/80_slices.yml | 4 ++++ .../rest-api-spec/test/update_by_query/70_slices.yml | 4 ++++ 4 files changed, 15 insertions(+) diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml index ad400ec718c87..127eeddbd8b11 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml @@ -95,6 +95,7 @@ index: test body: { "text": "test" } - do: + indices.refresh: {} - do: @@ -121,6 +122,8 @@ task_id: $task - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" indices.refresh: {} - do: diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml index 9f45dcb07a4c1..87effd85b3243 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml @@ -202,6 +202,8 @@ id: 6 body: { "text": "test" } - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" indices.refresh: {} # Start the task with a requests_per_second that should make it take a very long time @@ -265,6 +267,8 @@ # Only the "parent" reindex task wrote its status to the tasks index though - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" indices.refresh: {} - do: warnings: diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml index 4d4af5dd0a8d8..52d40aa9469ff 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/reindex/80_slices.yml @@ -162,6 +162,8 @@ # Only the "parent" reindex task wrote its status to the tasks index though - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" indices.refresh: {} - do: warnings: @@ -278,6 +280,8 @@ # Only the "parent" reindex task wrote its status to the tasks index though - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" indices.refresh: {} - do: warnings: diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml index 000693937ce7f..234d2d712efb3 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/update_by_query/70_slices.yml @@ -145,6 +145,8 @@ # Only the "parent" reindex task wrote its status to the tasks index though - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" indices.refresh: {} - do: warnings: @@ -252,6 +254,8 @@ # Only the "parent" reindex task wrote its status to the tasks index though - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" indices.refresh: {} - do: warnings: From d7b0f8e7ba50f39cbedd2748ff95fafe48dfff38 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 13:11:04 -0600 Subject: [PATCH 145/161] Properly populate client in RestControllerTests now that it's actually used --- .../rest/RestControllerTests.java | 61 +++++++++++-------- 1 file changed, 35 insertions(+), 26 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java b/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java index 8c8e83e5bfe42..af7fa45e17b2a 100644 --- a/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java @@ -41,8 +41,10 @@ import org.elasticsearch.http.HttpStats; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.usage.UsageService; +import org.junit.After; import org.junit.Before; import java.io.IOException; @@ -76,6 +78,7 @@ public class RestControllerTests extends ESTestCase { private RestController restController; private HierarchyCircuitBreakerService circuitBreakerService; private UsageService usageService; + private NodeClient client; @Before public void setup() { @@ -92,7 +95,8 @@ public void setup() { inFlightRequestsBreaker = circuitBreakerService.getBreaker(CircuitBreaker.IN_FLIGHT_REQUESTS); HttpServerTransport httpServerTransport = new TestHttpServerTransport(); - restController = new RestController(Collections.emptySet(), null, null, circuitBreakerService, usageService); + client = new NoOpNodeClient(this.getTestName()); + restController = new RestController(Collections.emptySet(), null, client, circuitBreakerService, usageService); restController.registerHandler(RestRequest.Method.GET, "/", (request, channel, client) -> channel.sendResponse( new BytesRestResponse(RestStatus.OK, BytesRestResponse.TEXT_CONTENT_TYPE, BytesArray.EMPTY))); @@ -103,8 +107,13 @@ public void setup() { httpServerTransport.start(); } + @After + public void teardown() { + client.close(); + } + public void testApplyRelevantHeaders() throws Exception { - final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + final ThreadContext threadContext = client.threadPool().getThreadContext(); Set headers = new HashSet<>(Arrays.asList(new RestHeaderDefinition("header.1", true), new RestHeaderDefinition("header.2", true))); final RestController restController = new RestController(headers, null, null, circuitBreakerService, usageService); @@ -140,7 +149,7 @@ public MethodHandlers next() { } public void testRequestWithDisallowedMultiValuedHeader() { - final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + final ThreadContext threadContext = client.threadPool().getThreadContext(); Set headers = new HashSet<>(Arrays.asList(new RestHeaderDefinition("header.1", true), new RestHeaderDefinition("header.2", false))); final RestController restController = new RestController(headers, null, null, circuitBreakerService, usageService); @@ -154,10 +163,10 @@ public void testRequestWithDisallowedMultiValuedHeader() { } public void testRequestWithDisallowedMultiValuedHeaderButSameValues() { - final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + final ThreadContext threadContext = client.threadPool().getThreadContext(); Set headers = new HashSet<>(Arrays.asList(new RestHeaderDefinition("header.1", true), new RestHeaderDefinition("header.2", false))); - final RestController restController = new RestController(headers, null, null, circuitBreakerService, usageService); + final RestController restController = new RestController(headers, null, client, circuitBreakerService, usageService); Map> restHeaders = new HashMap<>(); restHeaders.put("header.1", Collections.singletonList("boo")); restHeaders.put("header.2", List.of("foo", "foo")); @@ -238,11 +247,11 @@ public void testRestHandlerWrapper() throws Exception { h -> { assertSame(handler, h); return (RestRequest request, RestChannel channel, NodeClient client) -> wrapperCalled.set(true); - }, null, circuitBreakerService, usageService); + }, client, circuitBreakerService, usageService); restController.registerHandler(RestRequest.Method.GET, "/wrapped", handler); RestRequest request = testRestRequest("/wrapped", "{}", XContentType.JSON); AssertingChannel channel = new AssertingChannel(request, true, RestStatus.BAD_REQUEST); - restController.dispatchRequest(request, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(request, channel, client.threadPool().getThreadContext()); httpServerTransport.start(); assertTrue(wrapperCalled.get()); assertFalse(handlerCalled.get()); @@ -254,7 +263,7 @@ public void testDispatchRequestAddsAndFreesBytesOnSuccess() { RestRequest request = testRestRequest("/", content, XContentType.JSON); AssertingChannel channel = new AssertingChannel(request, true, RestStatus.OK); - restController.dispatchRequest(request, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(request, channel, client.threadPool().getThreadContext()); assertEquals(0, inFlightRequestsBreaker.getTrippedCount()); assertEquals(0, inFlightRequestsBreaker.getUsed()); @@ -266,7 +275,7 @@ public void testDispatchRequestAddsAndFreesBytesOnError() { RestRequest request = testRestRequest("/error", content, XContentType.JSON); AssertingChannel channel = new AssertingChannel(request, true, RestStatus.BAD_REQUEST); - restController.dispatchRequest(request, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(request, channel, client.threadPool().getThreadContext()); assertEquals(0, inFlightRequestsBreaker.getTrippedCount()); assertEquals(0, inFlightRequestsBreaker.getUsed()); @@ -279,7 +288,7 @@ public void testDispatchRequestAddsAndFreesBytesOnlyOnceOnError() { RestRequest request = testRestRequest("/error", content, XContentType.JSON); ExceptionThrowingChannel channel = new ExceptionThrowingChannel(request, true); - restController.dispatchRequest(request, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(request, channel, client.threadPool().getThreadContext()); assertEquals(0, inFlightRequestsBreaker.getTrippedCount()); assertEquals(0, inFlightRequestsBreaker.getUsed()); @@ -291,7 +300,7 @@ public void testDispatchRequestLimitsBytes() { RestRequest request = testRestRequest("/", content, XContentType.JSON); AssertingChannel channel = new AssertingChannel(request, true, RestStatus.TOO_MANY_REQUESTS); - restController.dispatchRequest(request, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(request, channel, client.threadPool().getThreadContext()); assertEquals(1, inFlightRequestsBreaker.getTrippedCount()); assertEquals(0, inFlightRequestsBreaker.getUsed()); @@ -307,7 +316,7 @@ public void testDispatchRequiresContentTypeForRequestsWithContent() { new BytesRestResponse(RestStatus.OK, BytesRestResponse.TEXT_CONTENT_TYPE, BytesArray.EMPTY))); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(request, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(request, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); } @@ -316,7 +325,7 @@ public void testDispatchDoesNotRequireContentTypeForRequestsWithoutContent() { AssertingChannel channel = new AssertingChannel(fakeRestRequest, true, RestStatus.OK); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); } @@ -334,7 +343,7 @@ public void handleRequest(RestRequest request, RestChannel channel, NodeClient c }); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); } @@ -345,7 +354,7 @@ public void testDispatchUnsupportedContentType() { AssertingChannel channel = new AssertingChannel(fakeRestRequest, true, RestStatus.NOT_ACCEPTABLE); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); } @@ -369,7 +378,7 @@ public boolean supportsContentStream() { }); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); } @@ -394,7 +403,7 @@ public boolean supportsContentStream() { }); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); } @@ -415,7 +424,7 @@ public boolean supportsContentStream() { }); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); } @@ -436,7 +445,7 @@ public boolean supportsContentStream() { } }); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); } @@ -458,7 +467,7 @@ public boolean supportsContentStream() { } }); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); } @@ -467,7 +476,7 @@ public void testDispatchBadRequest() { final AssertingChannel channel = new AssertingChannel(fakeRestRequest, true, RestStatus.BAD_REQUEST); restController.dispatchBadRequest( channel, - new ThreadContext(Settings.EMPTY), + client.threadPool().getThreadContext(), randomBoolean() ? new IllegalStateException("bad request") : new Throwable("bad request")); assertTrue(channel.getSendResponseCalled()); assertThat(channel.getRestResponse().content().utf8ToString(), containsString("bad request")); @@ -499,7 +508,7 @@ public boolean canTripCircuitBreaker() { assertFalse(channel.getSendResponseCalled()); assertFalse(restRequest.isContentConsumed()); - restController.dispatchRequest(restRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(restRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); assertFalse("RestController must not consume request content", restRequest.isContentConsumed()); @@ -508,7 +517,7 @@ public boolean canTripCircuitBreaker() { public void testDispatchBadRequestUnknownCause() { final FakeRestRequest fakeRestRequest = new FakeRestRequest.Builder(NamedXContentRegistry.EMPTY).build(); final AssertingChannel channel = new AssertingChannel(fakeRestRequest, true, RestStatus.BAD_REQUEST); - restController.dispatchBadRequest(channel, new ThreadContext(Settings.EMPTY), null); + restController.dispatchBadRequest(channel, client.threadPool().getThreadContext(), null); assertTrue(channel.getSendResponseCalled()); assertThat(channel.getRestResponse().content().utf8ToString(), containsString("unknown cause")); } @@ -519,7 +528,7 @@ public void testFavicon() { .withPath("/favicon.ico") .build(); final AssertingChannel channel = new AssertingChannel(fakeRestRequest, false, RestStatus.OK); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); assertThat(channel.getRestResponse().contentType(), containsString("image/x-icon")); } @@ -531,7 +540,7 @@ public void testFaviconWithWrongHttpMethod() { .withPath("/favicon.ico") .build(); final AssertingChannel channel = new AssertingChannel(fakeRestRequest, true, RestStatus.METHOD_NOT_ALLOWED); - restController.dispatchRequest(fakeRestRequest, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(fakeRestRequest, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); assertThat(channel.getRestResponse().getHeaders().containsKey("Allow"), equalTo(true)); assertThat(channel.getRestResponse().getHeaders().get("Allow"), hasItem(equalTo(RestRequest.Method.GET.toString()))); @@ -604,7 +613,7 @@ public Exception getInboundException() { final AssertingChannel channel = new AssertingChannel(request, true, RestStatus.METHOD_NOT_ALLOWED); assertFalse(channel.getSendResponseCalled()); - restController.dispatchRequest(request, channel, new ThreadContext(Settings.EMPTY)); + restController.dispatchRequest(request, channel, client.threadPool().getThreadContext()); assertTrue(channel.getSendResponseCalled()); assertThat(channel.getRestResponse().getHeaders().containsKey("Allow"), equalTo(true)); assertThat(channel.getRestResponse().getHeaders().get("Allow"), hasItem(equalTo(RestRequest.Method.GET.toString()))); From 45cf15d2823053b63c8eb0dd5ed1edfb9710afec Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 14:00:04 -0600 Subject: [PATCH 146/161] Stash context in RestSubmitAsyncSearchActionTests so the same headers don't get added multiple times --- .../xpack/search/RestSubmitAsyncSearchActionTests.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/RestSubmitAsyncSearchActionTests.java b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/RestSubmitAsyncSearchActionTests.java index 54e297cf25fba..e51c8a1054c9c 100644 --- a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/RestSubmitAsyncSearchActionTests.java +++ b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/RestSubmitAsyncSearchActionTests.java @@ -8,6 +8,7 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.test.rest.FakeRestRequest; @@ -96,7 +97,10 @@ private void doTestParameter(String paramName, String paramValue, T expected .withParams(params) .withContent(new BytesArray("{}"), XContentType.JSON).build(); - dispatchRequest(submitAsyncRestRequest); + // Get a new context each time, so we don't get exceptions due to trying to add the same header multiple times + try (ThreadContext.StoredContext context = verifyingClient.threadPool().getThreadContext().stashContext()) { + dispatchRequest(submitAsyncRestRequest); + } assertThat(executeCalled.get(), equalTo(true)); verifyingClient.reset(); } From 49bf8c5b7964f5f486f6b3439de2fcbd7ab75b31 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 14:19:57 -0600 Subject: [PATCH 147/161] Handle more _refresh warnings --- .../java/org/elasticsearch/upgrades/FullClusterRestartIT.java | 4 ++-- .../main/java/org/elasticsearch/test/rest/ESRestTestCase.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index 17cc250814327..26005d6921661 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -306,7 +306,7 @@ public void testShrink() throws IOException { shrinkIndexRequest.setJsonEntity("{\"settings\": {\"index.number_of_shards\": 1}}"); client().performRequest(shrinkIndexRequest); - client().performRequest(new Request("POST", "/_refresh")); + refreshAllIndices(); } else { numDocs = countOfIndexedRandomDocuments(); } @@ -382,7 +382,7 @@ public void testShrinkAfterUpgrade() throws IOException { numDocs = countOfIndexedRandomDocuments(); } - client().performRequest(new Request("POST", "/_refresh")); + refreshAllIndices(); Map response = entityAsMap(client().performRequest(new Request("GET", "/" + index + "/_search"))); assertNoFailures(response); diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 6fcd5bb02af7d..ec81c4c268319 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -812,7 +812,7 @@ private void wipeRollupJobs() throws IOException { protected void refreshAllIndices() throws IOException { boolean includeHidden = minimumNodeVersion().onOrAfter(Version.V_7_7_0); Request refreshRequest = new Request("POST", "/_refresh"); - refreshRequest.addParameter("expand_wildcards", "open,closed" + (includeHidden ? ",hidden" : "")); + refreshRequest.addParameter("expand_wildcards", "open" + (includeHidden ? ",hidden" : "")); // Allow system index deprecation warnings refreshRequest.setOptions(RequestOptions.DEFAULT.toBuilder().setWarningsHandler(warnings -> { if (warnings.isEmpty()) { From 18b98ce35ebfcbe34bc8bab048849d569761ae06 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 14:51:06 -0600 Subject: [PATCH 148/161] YAML skip: features: warnings block --- .../rest-api-spec/test/delete_by_query/70_throttle.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml index 127eeddbd8b11..a750980c70247 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml @@ -74,6 +74,8 @@ --- "Rethrottle to -1 which turns off throttling": + - skip: + - features: warnings # Throttling happens between each scroll batch so we need to control the size of the batch by using a single shard # and a small batch size on the request - do: From 2a6bbb2206ce3333b903f6a55bf41c3b22d51c4a Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 15:45:34 -0600 Subject: [PATCH 149/161] Address _refresh warnings in yaml tests --- .../resources/rest-api-spec/test/ml/delete_expired_data.yml | 4 ++++ .../test/resources/rest-api-spec/test/ml/filter_crud.yml | 3 +++ .../test/resources/rest-api-spec/test/ml/index_layout.yml | 5 +++++ .../rest-api-spec/test/ml/inference_stats_crud.yml | 5 ++++- .../src/test/resources/rest-api-spec/test/ml/jobs_crud.yml | 1 + .../test/resources/rest-api-spec/test/ml/jobs_get_stats.yml | 3 ++- .../rest-api-spec/test/security/authz/13_index_datemath.yml | 6 +++--- .../rest-api-spec/test/security/authz/15_auto_create.yml | 2 +- .../test/security/authz/31_rollover_using_alias.yml | 3 ++- .../resources/rest-api-spec/test/mixed_cluster/10_basic.yml | 3 ++- 10 files changed, 27 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/delete_expired_data.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/delete_expired_data.yml index 4cd5adafd020c..b4b9dbc03e3c1 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/delete_expired_data.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/delete_expired_data.yml @@ -71,6 +71,8 @@ setup: job_id: not-a-job --- "Test delete expired data with job id": + - skip: + features: warnings - do: headers: Content-Type: application/json @@ -153,6 +155,8 @@ setup: job_id: delete-expired-data-a - do: + warnings: + - "this request accesses system indices: [.ml-config], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" indices.refresh: {} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml index e6ae375261b0c..130171a818409 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/filter_crud.yml @@ -3,6 +3,7 @@ setup: - skip: features: - headers + - warnings - do: headers: @@ -26,6 +27,8 @@ setup: } - do: + warnings: + - "this request accesses system indices: [.ml-meta], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.refresh: {} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml index 2b8a9063b4ad3..64a613bd946d9 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/index_layout.yml @@ -118,6 +118,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.refresh: + index: [".ml-anomalies*", ".ml-state*"] expand_wildcards: all - do: @@ -478,6 +479,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.refresh: + index: ["foo", ".ml-anomalies*"] expand_wildcards: all - do: @@ -528,6 +530,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.refresh: + index: ".ml-state*" expand_wildcards: all - do: @@ -621,6 +624,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.refresh: + index: [".ml-state*", ".ml-anomalies*"] expand_wildcards: all - do: @@ -706,6 +710,7 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.refresh: + index: ".ml-state*" expand_wildcards: all - do: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml index 9d420c38bef68..2eb2430c2522c 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/inference_stats_crud.yml @@ -63,9 +63,12 @@ setup: } - do: + warnings: + - "this request accesses system indices: [.ml-inference-000003], but in a future major version, direct access to system indices will be prevented by default" headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser - indices.refresh: {} + indices.refresh: + index: ".ml-inference-*" - do: headers: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/jobs_crud.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/jobs_crud.yml index b8af6ad31d1f3..ec7528098db72 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/jobs_crud.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/jobs_crud.yml @@ -554,6 +554,7 @@ - do: indices.refresh: + index: ".ml-anomalies*" expand_wildcards: all - do: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/jobs_get_stats.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/jobs_get_stats.yml index 3ffd4087cb372..4313d48f0e146 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/jobs_get_stats.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/jobs_get_stats.yml @@ -267,7 +267,8 @@ setup: - do: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser - indices.refresh: {} + indices.refresh: + index: ".ml-anomalies*" # This is testing that the documents with v5.4 IDs are fetched. # Ideally we would use the v5.4 type but we can't put a mapping diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/13_index_datemath.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/13_index_datemath.yml index 462b023d18cc0..2651519e5f785 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/13_index_datemath.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/13_index_datemath.yml @@ -67,7 +67,7 @@ teardown: - do: # superuser indices.refresh: - index: "_all" + index: "write-*" - do: # superuser search: @@ -104,7 +104,7 @@ teardown: - do: # superuser indices.refresh: - index: "_all" + index: "read-*" - do: # superuser search: @@ -129,7 +129,7 @@ teardown: - do: # superuser indices.refresh: - index: "_all" + index: "write-*" - do: # superuser search: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/15_auto_create.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/15_auto_create.yml index bbe6f42f8270f..77948a95a9665 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/15_auto_create.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/15_auto_create.yml @@ -59,7 +59,7 @@ teardown: - do: # superuser indices.refresh: - index: "_all" + index: "logs-*" - do: # superuser search: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml index 52b6259f7ccf0..7983178ebb0f6 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml @@ -90,7 +90,8 @@ teardown: } - do: - indices.refresh: {} + indices.refresh: + index: "write_manage_alias" # rollover using alias - do: diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/10_basic.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/10_basic.yml index dd0639d0a65db..265f3547b6d65 100644 --- a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/10_basic.yml +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/10_basic.yml @@ -25,7 +25,8 @@ body: { foo: 2 } - do: - indices.refresh: {} + indices.refresh: + index: upgraded_scroll - do: search: From 5c92e966bd45ad7070be078cc4dd0e3c761fbe74 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 16:05:25 -0600 Subject: [PATCH 150/161] Warn when accessing aliases with names reserved for system indices --- .../alias/get/TransportGetAliasesAction.java | 39 ++++++++++++++----- .../get/TransportGetAliasesActionTests.java | 39 +++++++++++++++---- 2 files changed, 61 insertions(+), 17 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java index 925faf3e02083..1ce268c177f09 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java @@ -33,25 +33,31 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.indices.SystemIndices; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.stream.Collectors; public class TransportGetAliasesAction extends TransportMasterNodeReadAction { private static DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(TransportGetAliasesAction.class); + private final SystemIndices systemIndices; + @Inject public TransportGetAliasesAction(TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver) { + IndexNameExpressionResolver indexNameExpressionResolver, SystemIndices systemIndices) { super(GetAliasesAction.NAME, transportService, clusterService, threadPool, actionFilters, GetAliasesRequest::new, indexNameExpressionResolver); + this.systemIndices = systemIndices; } @Override @@ -80,9 +86,10 @@ protected void masterOperation(Task task, GetAliasesRequest request, ClusterStat try (ThreadContext.StoredContext ignore = threadPool.getThreadContext().newStoredContext(false)) { concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request); } + final boolean systemIndexAccessAllowed = indexNameExpressionResolver.isSystemIndexAccessAllowed(); ImmutableOpenMap> aliases = state.metadata().findAliases(request, concreteIndices); listener.onResponse(new GetAliasesResponse(postProcess(request, concreteIndices, aliases, state, - indexNameExpressionResolver.isSystemIndexAccessAllowed()))); + systemIndexAccessAllowed, systemIndices))); } /** @@ -90,7 +97,8 @@ protected void masterOperation(Task task, GetAliasesRequest request, ClusterStat */ static ImmutableOpenMap> postProcess(GetAliasesRequest request, String[] concreteIndices, ImmutableOpenMap> aliases, - ClusterState state, boolean systemIndexAccessAllowed) { + ClusterState state, boolean systemIndexAccessAllowed, + SystemIndices systemIndices) { boolean noAliasesSpecified = request.getOriginalAliases() == null || request.getOriginalAliases().length == 0; ImmutableOpenMap.Builder> mapBuilder = ImmutableOpenMap.builder(aliases); for (String index : concreteIndices) { @@ -101,25 +109,38 @@ static ImmutableOpenMap> postProcess(GetAliasesReque } final ImmutableOpenMap> finalResponse = mapBuilder.build(); if (systemIndexAccessAllowed == false) { - checkSystemIndexAccess(state, finalResponse); + checkSystemIndexAccess(request, systemIndices, state, finalResponse); } return finalResponse; } - private static void checkSystemIndexAccess(ClusterState state, ImmutableOpenMap> aliasesMap) { - List systemIndices = new ArrayList<>(); + private static void checkSystemIndexAccess(GetAliasesRequest request, SystemIndices systemIndices, ClusterState state, + ImmutableOpenMap> aliasesMap) { + List systemIndicesNames = new ArrayList<>(); for (Iterator it = aliasesMap.keysIt(); it.hasNext(); ) { String indexName = it.next(); IndexMetadata index = state.metadata().index(indexName); if (index != null && index.isSystem()) { - systemIndices.add(indexName); + systemIndicesNames.add(indexName); } } - if (systemIndices.isEmpty() == false) { + if (systemIndicesNames.isEmpty() == false) { deprecationLogger.deprecate("open_system_index_access", "this request accesses system indices: {}, but in a future major version, direct access to system " + - "indices will be prevented by default", systemIndices); + "indices will be prevented by default", systemIndicesNames); + } else { + checkSystemAliasAccess(request, systemIndices); } } + private static void checkSystemAliasAccess(GetAliasesRequest request, SystemIndices systemIndices) { + final List systemAliases = Arrays.stream(request.aliases()) + .filter(alias -> systemIndices.isSystemIndex(alias)) + .collect(Collectors.toList()); + if (systemAliases.isEmpty() == false) { + deprecationLogger.deprecate("open_system_alias_access", + "this request accesses aliases with names reserved for system indices: {}, but in a future major version, direct" + + "access to system indices and their aliases will not be allowed", systemAliases); + } + } } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java index 8960f663ccb5d..ce5961cc450b1 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java @@ -24,6 +24,8 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.indices.SystemIndexDescriptor; +import org.elasticsearch.indices.SystemIndices; import org.elasticsearch.test.ESTestCase; import java.util.Collections; @@ -32,6 +34,7 @@ import static org.hamcrest.Matchers.equalTo; public class TransportGetAliasesActionTests extends ESTestCase { + private final SystemIndices EMPTY_SYSTEM_INDICES = new SystemIndices(Collections.EMPTY_MAP); public void testPostProcess() { GetAliasesRequest request = new GetAliasesRequest(); @@ -39,7 +42,8 @@ public void testPostProcess() { .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) .build(); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE, false); + TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE, false, + EMPTY_SYSTEM_INDICES); assertThat(result.size(), equalTo(3)); assertThat(result.get("a").size(), equalTo(0)); assertThat(result.get("b").size(), equalTo(1)); @@ -50,7 +54,8 @@ public void testPostProcess() { aliases = ImmutableOpenMap.>builder() .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) .build(); - result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE, false); + result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE, false, + EMPTY_SYSTEM_INDICES); assertThat(result.size(), equalTo(3)); assertThat(result.get("a").size(), equalTo(0)); assertThat(result.get("b").size(), equalTo(1)); @@ -60,7 +65,8 @@ public void testPostProcess() { aliases = ImmutableOpenMap.>builder() .fPut("b", Collections.singletonList(new AliasMetadata.Builder("y").build())) .build(); - result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE, false); + result = TransportGetAliasesAction.postProcess(request, new String[]{"a", "b", "c"}, aliases, ClusterState.EMPTY_STATE, false, + EMPTY_SYSTEM_INDICES); assertThat(result.size(), equalTo(1)); assertThat(result.get("b").size(), equalTo(1)); } @@ -76,7 +82,7 @@ public void testDeprecationWarningEmittedForTotalWildcard() { final String[] concreteIndices = {"a", ".b", "c"}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false); + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false, EMPTY_SYSTEM_INDICES); assertThat(result.size(), equalTo(3)); assertThat(result.get("a").size(), equalTo(0)); assertThat(result.get(".b").size(), equalTo(1)); @@ -96,7 +102,7 @@ public void testDeprecationWarningEmittedWhenSystemIndexIsRequested() { final String[] concreteIndices = {".b"}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false); + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false, EMPTY_SYSTEM_INDICES); assertThat(result.size(), equalTo(1)); assertThat(result.get(".b").size(), equalTo(1)); assertWarnings("this request accesses system indices: [.b], but in a future major version, direct access to system " + @@ -113,7 +119,7 @@ public void testDeprecationWarningEmittedWhenSystemIndexIsRequestedByAlias() { final String[] concreteIndices = {"a", ".b", "c"}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false); + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false, EMPTY_SYSTEM_INDICES); assertThat(result.size(), equalTo(1)); assertThat(result.get(".b").size(), equalTo(1)); assertWarnings("this request accesses system indices: [.b], but in a future major version, direct access to system " + @@ -130,7 +136,7 @@ public void testDeprecationWarningNotEmittedWhenSystemAccessAllowed() { final String[] concreteIndices = {"a", ".b", "c"}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, true); + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, true, EMPTY_SYSTEM_INDICES); assertThat(result.size(), equalTo(1)); assertThat(result.get(".b").size(), equalTo(1)); } @@ -149,11 +155,28 @@ public void testDeprecationWarningNotEmittedWhenOnlyNonsystemIndexRequested() { final String[] concreteIndices = {"c"}; assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); ImmutableOpenMap> result = - TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false); + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false, EMPTY_SYSTEM_INDICES); assertThat(result.size(), equalTo(1)); assertThat(result.get("c").size(), equalTo(1)); } + public void testDeprecationWarningEmittedWhenRequestingNonExistingAliasInSystemPattern() { + ClusterState state = systemIndexTestClusterState(); + SystemIndices systemIndices = new SystemIndices(Collections.singletonMap(this.getTestName(), + Collections.singletonList(new SystemIndexDescriptor(".y", "an index that doesn't exist")))); + + GetAliasesRequest request = new GetAliasesRequest(".y"); + ImmutableOpenMap> aliases = ImmutableOpenMap.>builder() + .build(); + final String[] concreteIndices = {}; + assertEquals(state.metadata().findAliases(request, concreteIndices), aliases); + ImmutableOpenMap> result = + TransportGetAliasesAction.postProcess(request, concreteIndices, aliases, state, false, systemIndices); + assertThat(result.size(), equalTo(0)); + assertWarnings("this request accesses aliases with names reserved for system indices: [.y], but in a future major version, direct" + + "access to system indices and their aliases will not be allowed"); + } + public ClusterState systemIndexTestClusterState() { return ClusterState.builder(ClusterState.EMPTY_STATE) .metadata(Metadata.builder() From c852f30cbdf45dd9e1bc60f2101a431826748181 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 16:07:29 -0600 Subject: [PATCH 151/161] Suppress warnings when `X-elastic-product-origin` header is present --- .../src/main/java/org/elasticsearch/rest/RestController.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index 0b6fabb54568e..1aa9f20f311c5 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -65,6 +65,7 @@ public class RestController implements HttpServerTransport.Dispatcher { private static final Logger logger = LogManager.getLogger(RestController.class); private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(RestController.class); + private static final String ELASTIC_PRODUCT_ORIGIN_HTTP_HEADER = "X-elastic-product-origin"; private static final BytesReference FAVICON_RESPONSE; @@ -246,7 +247,7 @@ private void dispatchRequest(RestRequest request, RestChannel channel, RestHandl if (handler.allowsUnsafeBuffers() == false) { request.ensureSafeBuffers(); } - if (handler.allowSystemIndexAccessByDefault() == false) { + if (handler.allowSystemIndexAccessByDefault() == false && request.header(ELASTIC_PRODUCT_ORIGIN_HTTP_HEADER) != null) { client.threadPool().getThreadContext().putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); } From 833215a719c177208be1eee71a64eded99907ecc Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 16:26:21 -0600 Subject: [PATCH 152/161] EMPTY_MAP -> emptyMap() --- .../admin/indices/alias/get/TransportGetAliasesActionTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java index ce5961cc450b1..5af88f6fc768c 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesActionTests.java @@ -34,7 +34,7 @@ import static org.hamcrest.Matchers.equalTo; public class TransportGetAliasesActionTests extends ESTestCase { - private final SystemIndices EMPTY_SYSTEM_INDICES = new SystemIndices(Collections.EMPTY_MAP); + private final SystemIndices EMPTY_SYSTEM_INDICES = new SystemIndices(Collections.emptyMap()); public void testPostProcess() { GetAliasesRequest request = new GetAliasesRequest(); From f7c9e5b8bab6d089c01cb98850db1054a13369f3 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 16:37:03 -0600 Subject: [PATCH 153/161] Invert incorrect header check --- server/src/main/java/org/elasticsearch/rest/RestController.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index 1aa9f20f311c5..67be6cdd84b41 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -247,7 +247,7 @@ private void dispatchRequest(RestRequest request, RestChannel channel, RestHandl if (handler.allowsUnsafeBuffers() == false) { request.ensureSafeBuffers(); } - if (handler.allowSystemIndexAccessByDefault() == false && request.header(ELASTIC_PRODUCT_ORIGIN_HTTP_HEADER) != null) { + if (handler.allowSystemIndexAccessByDefault() == false && request.header(ELASTIC_PRODUCT_ORIGIN_HTTP_HEADER) == null) { client.threadPool().getThreadContext().putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); } From c4e5ed8e1d4235442d3729523a183a143a4308cb Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 17:06:40 -0600 Subject: [PATCH 154/161] Add comment about header --- .../src/main/java/org/elasticsearch/rest/RestController.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/rest/RestController.java b/server/src/main/java/org/elasticsearch/rest/RestController.java index 67be6cdd84b41..e1ca179460794 100644 --- a/server/src/main/java/org/elasticsearch/rest/RestController.java +++ b/server/src/main/java/org/elasticsearch/rest/RestController.java @@ -248,6 +248,9 @@ private void dispatchRequest(RestRequest request, RestChannel channel, RestHandl request.ensureSafeBuffers(); } if (handler.allowSystemIndexAccessByDefault() == false && request.header(ELASTIC_PRODUCT_ORIGIN_HTTP_HEADER) == null) { + // The ELASTIC_PRODUCT_ORIGIN_HTTP_HEADER indicates that the request is coming from an Elastic product with a plan + // to move away from direct access to system indices, and thus deprecation warnings should not be emitted. + // This header is intended for internal use only. client.threadPool().getThreadContext().putHeader(SYSTEM_INDEX_ACCESS_CONTROL_HEADER_KEY, Boolean.FALSE.toString()); } From 9d5dc5fcb5522e088177504ffd6a6a9259d4fc06 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 17:08:09 -0600 Subject: [PATCH 155/161] YAML syntax is THE WORST --- .../rest-api-spec/test/delete_by_query/70_throttle.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml index a750980c70247..4e877765c7cf2 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/70_throttle.yml @@ -75,7 +75,7 @@ --- "Rethrottle to -1 which turns off throttling": - skip: - - features: warnings + features: warnings # Throttling happens between each scroll batch so we need to control the size of the batch by using a single shard # and a small batch size on the request - do: From 5211807da498a3bbc32110ecd42e756e9f06dd99 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 17:11:37 -0600 Subject: [PATCH 156/161] More specifying _refresh more closely in YAML tests --- .../test/security/authz/31_rollover_using_alias.yml | 3 ++- .../test/set_security_user/10_small_users_one_index.yml | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml index 7983178ebb0f6..fd9f6d1d46050 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/31_rollover_using_alias.yml @@ -128,7 +128,8 @@ teardown: } - do: - indices.refresh: {} + indices.refresh: + index: write_manage_alias # check alias points to the new index and the doc was indexed - do: diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/set_security_user/10_small_users_one_index.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/set_security_user/10_small_users_one_index.yml index 80a1ea12dec3d..7442c74a9eae6 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/set_security_user/10_small_users_one_index.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/set_security_user/10_small_users_one_index.yml @@ -116,7 +116,8 @@ teardown: } - do: - indices.refresh: {} + indices.refresh: + index: shared_logs # Joe searches: - do: From 1e4223bfab1a558a72af09fccb5ae1d0d6fa0713 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 17:55:31 -0600 Subject: [PATCH 157/161] Yet another _refresh call with a warning --- .../xpack/transform/integration/TransformUsageIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java index 7ecde25a7fb32..f8efe1d581c4d 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformUsageIT.java @@ -123,7 +123,7 @@ public void testUsage() throws Exception { } } // Refresh the index so that statistics are searchable - refreshIndex(TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME); + refreshAllIndices(); }, 60, TimeUnit.SECONDS); stopTransform("test_usage_continuous", false); From 4e6c6d2b7f721c94352e0afc74a0241ade907402 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 17:58:07 -0600 Subject: [PATCH 158/161] Fix a couple _refresh calls that got mixed up --- .../rest-api-spec/test/delete_by_query/80_slices.yml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml index 87effd85b3243..884e19c363be9 100644 --- a/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml +++ b/modules/reindex/src/yamlRestTest/resources/rest-api-spec/test/delete_by_query/80_slices.yml @@ -153,6 +153,8 @@ # Only the "parent" reindex task wrote its status to the tasks index though - do: + warnings: + - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" indices.refresh: {} - do: warnings: @@ -202,9 +204,8 @@ id: 6 body: { "text": "test" } - do: - warnings: - - "this request accesses system indices: [.tasks], but in a future major version, direct access to system indices will be prevented by default" - indices.refresh: {} + indices.refresh: + index: test # Start the task with a requests_per_second that should make it take a very long time - do: From b9342bb80019ba6967c98e7f90a6d81a5347c992 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 19:09:51 -0600 Subject: [PATCH 159/161] Even more _refresh index specifying --- .../rest-api-spec/test/10_templated_role_query.yml | 3 ++- .../rest-api-spec/test/11_templated_role_query_runas.yml | 3 ++- .../rest-api-spec/test/20_small_users_one_index.yml | 6 ++++-- .../resources/rest-api-spec/test/30_search_template.yml | 3 ++- 4 files changed, 10 insertions(+), 5 deletions(-) diff --git a/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/10_templated_role_query.yml b/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/10_templated_role_query.yml index 84d8d98e27384..4dcc8c847c464 100644 --- a/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/10_templated_role_query.yml +++ b/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/10_templated_role_query.yml @@ -125,7 +125,8 @@ setup: } - do: - indices.refresh: {} + indices.refresh: + index: foobar --- teardown: diff --git a/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/11_templated_role_query_runas.yml b/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/11_templated_role_query_runas.yml index 2f4755943aa2d..b3948028f4144 100644 --- a/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/11_templated_role_query_runas.yml +++ b/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/11_templated_role_query_runas.yml @@ -125,7 +125,8 @@ setup: } - do: - indices.refresh: {} + indices.refresh: + index: foobar --- teardown: diff --git a/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/20_small_users_one_index.yml b/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/20_small_users_one_index.yml index e3f706570a22a..4e38838f5dce1 100644 --- a/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/20_small_users_one_index.yml +++ b/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/20_small_users_one_index.yml @@ -107,7 +107,8 @@ teardown: } - do: - indices.refresh: {} + indices.refresh: + index: shared_logs # Joe searches: - do: @@ -177,7 +178,8 @@ teardown: } - do: - indices.refresh: {} + indices.refresh: + index: shared_logs # Joe searches: - do: diff --git a/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/30_search_template.yml b/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/30_search_template.yml index a208bda67cfe2..1ce18208a1085 100644 --- a/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/30_search_template.yml +++ b/x-pack/qa/smoke-test-security-with-mustache/src/test/resources/rest-api-spec/test/30_search_template.yml @@ -44,7 +44,8 @@ setup: title: "contains some words too" - do: - indices.refresh: {} + indices.refresh: + index: ["foobar", "unauthorized_index"] --- teardown: From 0ea0be50bf135f020ef5c49506dd88e0a390a1be Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Mon, 5 Oct 2020 20:00:47 -0600 Subject: [PATCH 160/161] Another _refresh that sparks a warning --- .../integration/TransformConfigurationIndexIT.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java index c04d2ec72e47a..68c8e249c1b99 100644 --- a/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java +++ b/x-pack/plugin/transform/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformConfigurationIndexIT.java @@ -9,6 +9,7 @@ import org.apache.http.entity.ContentType; import org.apache.http.entity.StringEntity; import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.common.Strings; @@ -33,6 +34,9 @@ public class TransformConfigurationIndexIT extends TransformRestTestCase { */ public void testDeleteConfigurationLeftOver() throws IOException { String fakeTransformName = randomAlphaOfLengthBetween(5, 20); + final RequestOptions expectWarningOptions = expectWarnings("this request accesses system indices: [" + + TransformInternalIndexConstants.LATEST_INDEX_NAME + "], but in a future major version, direct access to system indices will " + + "be prevented by default"); try (XContentBuilder builder = jsonBuilder()) { builder.startObject(); @@ -43,14 +47,15 @@ public void testDeleteConfigurationLeftOver() throws IOException { final StringEntity entity = new StringEntity(Strings.toString(builder), ContentType.APPLICATION_JSON); Request req = new Request("PUT", TransformInternalIndexConstants.LATEST_INDEX_NAME + "/_doc/" + TransformConfig.documentId(fakeTransformName)); - req.setOptions(expectWarnings("this request accesses system indices: [" + TransformInternalIndexConstants.LATEST_INDEX_NAME + - "], but in a future major version, direct access to system indices will be prevented by default")); + req.setOptions(expectWarningOptions); req.setEntity(entity); client().performRequest(req); } // refresh the index - assertOK(client().performRequest(new Request("POST", TransformInternalIndexConstants.LATEST_INDEX_NAME + "/_refresh"))); + final Request refreshRequest = new Request("POST", TransformInternalIndexConstants.LATEST_INDEX_NAME + "/_refresh"); + refreshRequest.setOptions(expectWarningOptions); + assertOK(client().performRequest(refreshRequest)); Request deleteRequest = new Request("DELETE", getTransformEndpoint() + fakeTransformName); Response deleteResponse = client().performRequest(deleteRequest); From dac5c74a15e38a57fd5ab16db285a3ae04ff1f66 Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Tue, 6 Oct 2020 09:56:01 -0600 Subject: [PATCH 161/161] Review feedback --- .../admin/indices/alias/get/TransportGetAliasesAction.java | 2 +- .../java/org/elasticsearch/rest/RestControllerTests.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java index 1ce268c177f09..67fcd04b0c72b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/alias/get/TransportGetAliasesAction.java @@ -47,7 +47,7 @@ import java.util.stream.Collectors; public class TransportGetAliasesAction extends TransportMasterNodeReadAction { - private static DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(TransportGetAliasesAction.class); + private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(TransportGetAliasesAction.class); private final SystemIndices systemIndices; diff --git a/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java b/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java index af7fa45e17b2a..0b756acb5ef86 100644 --- a/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/RestControllerTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.yaml.YamlXContent; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.http.HttpInfo; import org.elasticsearch.http.HttpRequest; import org.elasticsearch.http.HttpResponse; @@ -108,8 +109,8 @@ public void setup() { } @After - public void teardown() { - client.close(); + public void teardown() throws IOException { + IOUtils.close(client); } public void testApplyRelevantHeaders() throws Exception {