Add selector syntax to index expressions (#118614)

This PR introduces a new syntactical feature to index expression resolution: The selector.

Selectors, denoted with a :: followed by a recognized suffix will allow users to specify which component of 
an index abstraction they would like to operate on within an API call. In this case, an index abstraction is a 
concrete index, data stream, or alias; Any abstraction that can be resolved to a set of indices/shards. We 
define a component of an index abstraction to be some searchable unit of the index abstraction.
This commit is contained in:
James Baiera 2025-01-08 11:37:03 -05:00 committed by GitHub
parent 82b1f2a205
commit c3839e1f76
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
73 changed files with 2377 additions and 1073 deletions

View file

@ -557,4 +557,3 @@ The API returns the following results:
// TESTRESPONSE[s/"job_version" : "8.4.0"/"job_version" : $body.job_version/]
// TESTRESPONSE[s/1656087283340/$body.$_path/]
// TESTRESPONSE[s/"superuser"/"_es_test_root"/]
// TESTRESPONSE[s/"ignore_throttled" : true/"ignore_throttled" : true,"failure_store":"exclude"/]

View file

@ -20,6 +20,7 @@ restResources {
dependencies {
testImplementation project(path: ':test:test-clusters')
testImplementation project(":modules:mapper-extras")
internalClusterTestImplementation project(":modules:mapper-extras")
}
@ -70,4 +71,16 @@ tasks.named("yamlRestCompatTestTransform").configure({ task ->
task.skipTest("data_stream/200_rollover_failure_store/Lazily roll over a data stream's failure store after a shard failure", "Configuring the failure store via data stream templates is not supported anymore.")
task.skipTest("data_stream/200_rollover_failure_store/Don't roll over a data stream's failure store when conditions aren't met", "Configuring the failure store via data stream templates is not supported anymore.")
task.skipTest("data_stream/200_rollover_failure_store/Roll over a data stream's failure store with conditions", "Configuring the failure store via data stream templates is not supported anymore.")
task.skipTest("data_stream/200_rollover_failure_store/Rolling over a failure store on a data stream without the failure store enabled should work", "Rolling over a data stream using target_failure_store is no longer supported.")
task.skipTest("data_stream/200_rollover_failure_store/Rolling over an uninitialized failure store should initialize it", "Rolling over a data stream using target_failure_store is no longer supported.")
task.skipTest("data_stream/210_rollover_failure_store/A failure store marked for lazy rollover should only be rolled over when there is a failure", "Rolling over a data stream using target_failure_store is no longer supported.")
task.skipTest("data_stream/210_rollover_failure_store/Don't roll over a data stream's failure store when conditions aren't met", "Rolling over a data stream using target_failure_store is no longer supported.")
task.skipTest("data_stream/210_rollover_failure_store/Rolling over a failure store on a data stream without the failure store enabled should work", "Rolling over a data stream using target_failure_store is no longer supported.")
task.skipTest("data_stream/210_rollover_failure_store/Rolling over an uninitialized failure store should initialize it", "Rolling over a data stream using target_failure_store is no longer supported.")
task.skipTest("data_stream/210_rollover_failure_store/Roll over a data stream's failure store with conditions", "Rolling over a data stream using target_failure_store is no longer supported.")
task.skipTest("data_stream/210_rollover_failure_store/Lazily roll over a data stream's failure store after an ingest failure", "Rolling over a data stream using target_failure_store is no longer supported.")
task.skipTest("data_stream/210_rollover_failure_store/Lazily roll over a data stream's failure store after a shard failure", "Rolling over a data stream using target_failure_store is no longer supported.")
task.skipTest("data_stream/210_rollover_failure_store/Roll over a data stream's failure store without conditions", "Rolling over a data stream using target_failure_store is no longer supported.")
})

View file

@ -31,11 +31,13 @@ import org.elasticsearch.action.admin.indices.template.delete.TransportDeleteCom
import org.elasticsearch.action.datastreams.CreateDataStreamAction;
import org.elasticsearch.action.datastreams.DeleteDataStreamAction;
import org.elasticsearch.action.datastreams.GetDataStreamAction;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.DataStreamAlias;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.index.Index;
@ -136,10 +138,7 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
assertTrue(response.isAcknowledged());
// Initialize the failure store.
RolloverRequest rolloverRequest = new RolloverRequest("with-fs", null);
rolloverRequest.setIndicesOptions(
IndicesOptions.builder(rolloverRequest.indicesOptions()).selectorOptions(IndicesOptions.SelectorOptions.FAILURES).build()
);
RolloverRequest rolloverRequest = new RolloverRequest("with-fs::failures", null);
response = client.execute(RolloverAction.INSTANCE, rolloverRequest).get();
assertTrue(response.isAcknowledged());
@ -345,7 +344,7 @@ public class DataStreamsSnapshotsIT extends AbstractSnapshotIntegTestCase {
.cluster()
.prepareCreateSnapshot(TEST_REQUEST_TIMEOUT, REPO, SNAPSHOT)
.setWaitForCompletion(true)
.setIndices(dataStreamName)
.setIndices(IndexNameExpressionResolver.combineSelector(dataStreamName, IndexComponentSelector.ALL_APPLICABLE))
.setIncludeGlobalState(false)
.get();

View file

@ -20,11 +20,12 @@ import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.FailureStoreMetrics;
import org.elasticsearch.action.datastreams.CreateDataStreamAction;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.Template;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.core.Strings;
@ -194,9 +195,9 @@ public class IngestFailureStoreMetricsIT extends ESIntegTestCase {
createDataStream();
// Initialize failure store.
var rolloverRequest = new RolloverRequest(dataStream, null);
rolloverRequest.setIndicesOptions(
IndicesOptions.builder(rolloverRequest.indicesOptions()).selectorOptions(IndicesOptions.SelectorOptions.FAILURES).build()
var rolloverRequest = new RolloverRequest(
IndexNameExpressionResolver.combineSelector(dataStream, IndexComponentSelector.FAILURES),
null
);
var rolloverResponse = client().execute(RolloverAction.INSTANCE, rolloverRequest).actionGet();
var failureStoreIndex = rolloverResponse.getNewIndex();

View file

@ -60,7 +60,7 @@ public class DataStreamOptionsIT extends DisabledSecurityDataStreamTestCase {
assertOK(client().performRequest(new Request("PUT", "/_data_stream/" + DATA_STREAM_NAME)));
// Initialize the failure store.
assertOK(client().performRequest(new Request("POST", DATA_STREAM_NAME + "/_rollover?target_failure_store")));
assertOK(client().performRequest(new Request("POST", DATA_STREAM_NAME + "::failures/_rollover")));
ensureGreen(DATA_STREAM_NAME);
final Response dataStreamResponse = client().performRequest(new Request("GET", "/_data_stream/" + DATA_STREAM_NAME));

View file

@ -1,221 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/
package org.elasticsearch.datastreams;
import org.elasticsearch.client.Request;
import org.elasticsearch.client.Response;
import org.elasticsearch.client.ResponseException;
import org.junit.Before;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
/**
* This should be a yaml test, but in order to write one we would need to expose the new parameter in the rest-api-spec.
* We do not want to do that until the feature flag is removed. For this reason, we temporarily, test the affected APIs here.
* Please convert this to a yaml test when the feature flag is removed.
*/
public class FailureStoreQueryParamIT extends DisabledSecurityDataStreamTestCase {
private static final String DATA_STREAM_NAME = "failure-data-stream";
private String backingIndex;
private String failureStoreIndex;
@SuppressWarnings("unchecked")
@Before
public void setup() throws IOException {
Request putComposableIndexTemplateRequest = new Request("POST", "/_index_template/ds-template");
putComposableIndexTemplateRequest.setJsonEntity("""
{
"index_patterns": ["failure-data-stream"],
"template": {
"settings": {
"number_of_replicas": 0
},
"data_stream_options": {
"failure_store": {
"enabled": true
}
}
},
"data_stream": {
}
}
""");
assertOK(client().performRequest(putComposableIndexTemplateRequest));
assertOK(client().performRequest(new Request("PUT", "/_data_stream/" + DATA_STREAM_NAME)));
// Initialize the failure store.
assertOK(client().performRequest(new Request("POST", DATA_STREAM_NAME + "/_rollover?target_failure_store")));
ensureGreen(DATA_STREAM_NAME);
final Response dataStreamResponse = client().performRequest(new Request("GET", "/_data_stream/" + DATA_STREAM_NAME));
List<Object> dataStreams = (List<Object>) entityAsMap(dataStreamResponse).get("data_streams");
assertThat(dataStreams.size(), is(1));
Map<String, Object> dataStream = (Map<String, Object>) dataStreams.get(0);
assertThat(dataStream.get("name"), equalTo(DATA_STREAM_NAME));
List<String> backingIndices = getIndices(dataStream);
assertThat(backingIndices.size(), is(1));
List<String> failureStore = getFailureStore(dataStream);
assertThat(failureStore.size(), is(1));
backingIndex = backingIndices.get(0);
failureStoreIndex = failureStore.get(0);
}
public void testGetIndexApi() throws IOException {
{
final Response indicesResponse = client().performRequest(new Request("GET", "/" + DATA_STREAM_NAME));
Map<String, Object> indices = entityAsMap(indicesResponse);
assertThat(indices.size(), is(2));
assertThat(indices.containsKey(backingIndex), is(true));
assertThat(indices.containsKey(failureStoreIndex), is(true));
}
{
final Response indicesResponse = client().performRequest(new Request("GET", "/" + DATA_STREAM_NAME + "?failure_store=exclude"));
Map<String, Object> indices = entityAsMap(indicesResponse);
assertThat(indices.size(), is(1));
assertThat(indices.containsKey(backingIndex), is(true));
}
{
final Response indicesResponse = client().performRequest(new Request("GET", "/" + DATA_STREAM_NAME + "?failure_store=only"));
Map<String, Object> indices = entityAsMap(indicesResponse);
assertThat(indices.size(), is(1));
assertThat(indices.containsKey(failureStoreIndex), is(true));
}
}
@SuppressWarnings("unchecked")
public void testGetIndexStatsApi() throws IOException {
{
final Response statsResponse = client().performRequest(new Request("GET", "/" + DATA_STREAM_NAME + "/_stats"));
Map<String, Object> indices = (Map<String, Object>) entityAsMap(statsResponse).get("indices");
assertThat(indices.size(), is(1));
assertThat(indices.containsKey(backingIndex), is(true));
}
{
final Response statsResponse = client().performRequest(
new Request("GET", "/" + DATA_STREAM_NAME + "/_stats?failure_store=include")
);
Map<String, Object> indices = (Map<String, Object>) entityAsMap(statsResponse).get("indices");
assertThat(indices.size(), is(2));
assertThat(indices.containsKey(backingIndex), is(true));
assertThat(indices.containsKey(failureStoreIndex), is(true));
}
{
final Response statsResponse = client().performRequest(
new Request("GET", "/" + DATA_STREAM_NAME + "/_stats?failure_store=only")
);
Map<String, Object> indices = (Map<String, Object>) entityAsMap(statsResponse).get("indices");
assertThat(indices.size(), is(1));
assertThat(indices.containsKey(failureStoreIndex), is(true));
}
}
public void testGetIndexSettingsApi() throws IOException {
{
final Response indicesResponse = client().performRequest(new Request("GET", "/" + DATA_STREAM_NAME + "/_settings"));
Map<String, Object> indices = entityAsMap(indicesResponse);
assertThat(indices.size(), is(1));
assertThat(indices.containsKey(backingIndex), is(true));
}
{
final Response indicesResponse = client().performRequest(
new Request("GET", "/" + DATA_STREAM_NAME + "/_settings?failure_store=include")
);
Map<String, Object> indices = entityAsMap(indicesResponse);
assertThat(indices.size(), is(2));
assertThat(indices.containsKey(backingIndex), is(true));
assertThat(indices.containsKey(failureStoreIndex), is(true));
}
{
final Response indicesResponse = client().performRequest(
new Request("GET", "/" + DATA_STREAM_NAME + "/_settings?failure_store=only")
);
Map<String, Object> indices = entityAsMap(indicesResponse);
assertThat(indices.size(), is(1));
assertThat(indices.containsKey(failureStoreIndex), is(true));
}
}
public void testGetIndexMappingApi() throws IOException {
{
final Response indicesResponse = client().performRequest(new Request("GET", "/" + DATA_STREAM_NAME + "/_mapping"));
Map<String, Object> indices = entityAsMap(indicesResponse);
assertThat(indices.size(), is(1));
assertThat(indices.containsKey(backingIndex), is(true));
}
{
final Response indicesResponse = client().performRequest(
new Request("GET", "/" + DATA_STREAM_NAME + "/_mapping?failure_store=include")
);
Map<String, Object> indices = entityAsMap(indicesResponse);
assertThat(indices.size(), is(2));
assertThat(indices.containsKey(backingIndex), is(true));
assertThat(indices.containsKey(failureStoreIndex), is(true));
}
{
final Response indicesResponse = client().performRequest(
new Request("GET", "/" + DATA_STREAM_NAME + "/_mapping?failure_store=only")
);
Map<String, Object> indices = entityAsMap(indicesResponse);
assertThat(indices.size(), is(1));
assertThat(indices.containsKey(failureStoreIndex), is(true));
}
}
@SuppressWarnings("unchecked")
public void testPutIndexMappingApi() throws IOException {
{
final Request mappingRequest = new Request("PUT", "/" + DATA_STREAM_NAME + "/_mapping");
mappingRequest.setJsonEntity("""
{
"properties": {
"email": {
"type": "keyword"
}
}
}
""");
assertAcknowledged(client().performRequest(mappingRequest));
}
{
final Request mappingRequest = new Request("PUT", "/" + DATA_STREAM_NAME + "/_mapping?failure_store=include");
mappingRequest.setJsonEntity("""
{
"properties": {
"email": {
"type": "keyword"
}
}
}
""");
ResponseException responseException = expectThrows(ResponseException.class, () -> client().performRequest(mappingRequest));
Map<String, Object> response = entityAsMap(responseException.getResponse());
assertThat(((Map<String, Object>) response.get("error")).get("reason"), is("failure index not supported"));
}
}
@SuppressWarnings("unchecked")
private List<String> getFailureStore(Map<String, Object> response) {
var failureStore = (Map<String, Object>) response.get("failure_store");
return getIndices(failureStore);
}
@SuppressWarnings("unchecked")
private List<String> getIndices(Map<String, Object> response) {
List<Map<String, String>> indices = (List<Map<String, String>>) response.get("indices");
return indices.stream().map(index -> index.get("index_name")).toList();
}
}

View file

@ -16,6 +16,7 @@ import org.elasticsearch.action.datastreams.DataStreamsActionUtil;
import org.elasticsearch.action.datastreams.DataStreamsStatsAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
@ -102,10 +103,11 @@ public class DataStreamsStatsTransportAction extends TransportBroadcastByNodeAct
@Override
protected String[] resolveConcreteIndexNames(ClusterState clusterState, DataStreamsStatsAction.Request request) {
return DataStreamsActionUtil.resolveConcreteIndexNames(
return DataStreamsActionUtil.resolveConcreteIndexNamesWithSelector(
indexNameExpressionResolver,
clusterState,
request.indices(),
IndexComponentSelector.ALL_APPLICABLE,
request.indicesOptions()
).toArray(String[]::new);
}
@ -163,13 +165,17 @@ public class DataStreamsStatsTransportAction extends TransportBroadcastByNodeAct
request.indicesOptions(),
request.indices()
);
for (String abstractionName : abstractionNames) {
IndexAbstraction indexAbstraction = indicesLookup.get(abstractionName);
for (String abstraction : abstractionNames) {
IndexAbstraction indexAbstraction = indicesLookup.get(abstraction);
assert indexAbstraction != null;
if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM) {
DataStream dataStream = (DataStream) indexAbstraction;
AggregatedStats stats = aggregatedDataStreamsStats.computeIfAbsent(dataStream.getName(), s -> new AggregatedStats());
dataStream.getIndices().stream().map(Index::getName).forEach(index -> {
dataStream.getBackingIndices().getIndices().stream().map(Index::getName).forEach(index -> {
stats.backingIndices.add(index);
allBackingIndices.add(index);
});
dataStream.getFailureIndices().getIndices().stream().map(Index::getName).forEach(index -> {
stats.backingIndices.add(index);
allBackingIndices.add(index);
});

View file

@ -33,7 +33,7 @@ import org.elasticsearch.action.datastreams.lifecycle.ErrorEntry;
import org.elasticsearch.action.downsample.DownsampleAction;
import org.elasticsearch.action.downsample.DownsampleConfig;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client;
@ -49,6 +49,9 @@ import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionSettings;
import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SelectorResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.service.ClusterService;
@ -944,11 +947,6 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
if ((configuredFloorSegmentMerge == null || configuredFloorSegmentMerge.equals(targetMergePolicyFloorSegment) == false)
|| (configuredMergeFactor == null || configuredMergeFactor.equals(targetMergePolicyFactor) == false)) {
UpdateSettingsRequest updateMergePolicySettingsRequest = new UpdateSettingsRequest();
updateMergePolicySettingsRequest.indicesOptions(
IndicesOptions.builder(updateMergePolicySettingsRequest.indicesOptions())
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.build()
);
updateMergePolicySettingsRequest.indices(indexName);
updateMergePolicySettingsRequest.settings(
Settings.builder()
@ -998,8 +996,11 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
private void rolloverDataStream(String writeIndexName, RolloverRequest rolloverRequest, ActionListener<Void> listener) {
// "saving" the rollover target name here so we don't capture the entire request
String rolloverTarget = rolloverRequest.getRolloverTarget();
logger.trace("Data stream lifecycle issues rollover request for data stream [{}]", rolloverTarget);
ResolvedExpression resolvedRolloverTarget = SelectorResolver.parseExpression(
rolloverRequest.getRolloverTarget(),
rolloverRequest.indicesOptions()
);
logger.trace("Data stream lifecycle issues rollover request for data stream [{}]", rolloverRequest.getRolloverTarget());
client.admin().indices().rolloverIndex(rolloverRequest, new ActionListener<>() {
@Override
public void onResponse(RolloverResponse rolloverResponse) {
@ -1014,7 +1015,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
logger.info(
"Data stream lifecycle successfully rolled over datastream [{}] due to the following met rollover "
+ "conditions {}. The new index is [{}]",
rolloverTarget,
rolloverRequest.getRolloverTarget(),
metConditions,
rolloverResponse.getNewIndex()
);
@ -1024,7 +1025,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
@Override
public void onFailure(Exception e) {
DataStream dataStream = clusterService.state().metadata().dataStreams().get(rolloverTarget);
DataStream dataStream = clusterService.state().metadata().dataStreams().get(resolvedRolloverTarget.resource());
if (dataStream == null || dataStream.getWriteIndex().getName().equals(writeIndexName) == false) {
// the data stream has another write index so no point in recording an error for the previous write index we were
// attempting to roll over
@ -1407,9 +1408,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab
) {
RolloverRequest rolloverRequest = new RolloverRequest(dataStream, null).masterNodeTimeout(TimeValue.MAX_VALUE);
if (rolloverFailureStore) {
rolloverRequest.setIndicesOptions(
IndicesOptions.builder(rolloverRequest.indicesOptions()).selectorOptions(IndicesOptions.SelectorOptions.FAILURES).build()
);
rolloverRequest.setRolloverTarget(IndexNameExpressionResolver.combineSelector(dataStream, IndexComponentSelector.FAILURES));
}
rolloverRequest.setConditions(rolloverConfiguration.resolveRolloverConditions(dataRetention));
return rolloverRequest;

View file

@ -50,7 +50,7 @@ public class DeleteDataStreamLifecycleAction {
.allowAliasToMultipleIndices(false)
.allowClosedIndices(true)
.ignoreThrottled(false)
.allowFailureIndices(false)
.allowSelectors(false)
.build()
)
.build();

View file

@ -39,7 +39,9 @@ public class DeleteDataStreamOptionsAction {
.wildcardOptions(
IndicesOptions.WildcardOptions.builder().matchOpen(true).matchClosed(true).allowEmptyExpressions(true).resolveAliases(false)
)
.gatekeeperOptions(IndicesOptions.GatekeeperOptions.builder().allowAliasToMultipleIndices(false).allowClosedIndices(true))
.gatekeeperOptions(
IndicesOptions.GatekeeperOptions.builder().allowAliasToMultipleIndices(false).allowClosedIndices(true).allowSelectors(false)
)
.build();
public Request(StreamInput in) throws IOException {

View file

@ -50,7 +50,9 @@ public class GetDataStreamOptionsAction {
.wildcardOptions(
IndicesOptions.WildcardOptions.builder().matchOpen(true).matchClosed(true).allowEmptyExpressions(true).resolveAliases(false)
)
.gatekeeperOptions(IndicesOptions.GatekeeperOptions.builder().allowAliasToMultipleIndices(false).allowClosedIndices(true))
.gatekeeperOptions(
IndicesOptions.GatekeeperOptions.builder().allowAliasToMultipleIndices(false).allowClosedIndices(true).allowSelectors(false)
)
.build();
private boolean includeDefaults = false;

View file

@ -71,7 +71,9 @@ public class PutDataStreamOptionsAction {
.wildcardOptions(
IndicesOptions.WildcardOptions.builder().matchOpen(true).matchClosed(true).allowEmptyExpressions(true).resolveAliases(false)
)
.gatekeeperOptions(IndicesOptions.GatekeeperOptions.builder().allowAliasToMultipleIndices(false).allowClosedIndices(true))
.gatekeeperOptions(
IndicesOptions.GatekeeperOptions.builder().allowAliasToMultipleIndices(false).allowClosedIndices(true).allowSelectors(false)
)
.build();
private final DataStreamOptions options;

View file

@ -11,7 +11,6 @@ package org.elasticsearch.datastreams.rest;
import org.elasticsearch.action.datastreams.GetDataStreamAction;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.util.set.Sets;
@ -42,8 +41,7 @@ public class RestGetDataStreamsAction extends BaseRestHandler {
IndicesOptions.WildcardOptions.ALLOW_NO_INDICES,
IndicesOptions.GatekeeperOptions.IGNORE_THROTTLED,
"verbose"
),
DataStream.isFailureStoreFeatureFlagEnabled() ? Set.of(IndicesOptions.FAILURE_STORE_QUERY_PARAM) : Set.of()
)
)
);

View file

@ -15,6 +15,7 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.admin.indices.template.delete.TransportDeleteComposableIndexTemplateAction;
import org.elasticsearch.action.admin.indices.template.put.TransportPutComposableIndexTemplateAction;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.datastreams.CreateDataStreamAction;
import org.elasticsearch.action.datastreams.DataStreamsStatsAction;
import org.elasticsearch.action.datastreams.DeleteDataStreamAction;
@ -22,8 +23,12 @@ import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
import org.elasticsearch.cluster.metadata.DataStreamFailureStore;
import org.elasticsearch.cluster.metadata.DataStreamOptions;
import org.elasticsearch.cluster.metadata.ResettableValue;
import org.elasticsearch.cluster.metadata.Template;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.index.mapper.extras.MapperExtrasPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.xcontent.json.JsonXContent;
@ -40,12 +45,14 @@ import java.util.concurrent.TimeUnit;
import static java.lang.Math.max;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.is;
public class DataStreamsStatsTests extends ESSingleNodeTestCase {
@Override
protected Collection<Class<? extends Plugin>> getPlugins() {
return List.of(DataStreamsPlugin.class);
return List.of(DataStreamsPlugin.class, MapperExtrasPlugin.class);
}
private final Set<String> createdDataStreams = new HashSet<>();
@ -107,8 +114,30 @@ public class DataStreamsStatsTests extends ESSingleNodeTestCase {
assertEquals(stats.getTotalStoreSize().getBytes(), stats.getDataStreams()[0].getStoreSize().getBytes());
}
public void testStatsExistingDataStreamWithFailureStores() throws Exception {
String dataStreamName = createDataStream(false, true);
createFailedDocument(dataStreamName);
DataStreamsStatsAction.Response stats = getDataStreamsStats();
assertEquals(2, stats.getSuccessfulShards());
assertEquals(0, stats.getFailedShards());
assertEquals(1, stats.getDataStreamCount());
assertEquals(2, stats.getBackingIndices());
assertNotEquals(0L, stats.getTotalStoreSize().getBytes());
assertEquals(1, stats.getDataStreams().length);
assertEquals(dataStreamName, stats.getDataStreams()[0].getDataStream());
assertEquals(2, stats.getDataStreams()[0].getBackingIndices());
// The timestamp is going to not be something we can validate because
// it captures the time of failure which is uncontrolled in the test
// Just make sure it exists by ensuring it isn't zero
assertThat(stats.getDataStreams()[0].getMaximumTimestamp(), is(greaterThan(0L)));
assertNotEquals(0L, stats.getDataStreams()[0].getStoreSize().getBytes());
assertEquals(stats.getTotalStoreSize().getBytes(), stats.getDataStreams()[0].getStoreSize().getBytes());
}
public void testStatsExistingHiddenDataStream() throws Exception {
String dataStreamName = createDataStream(true);
String dataStreamName = createDataStream(true, false);
long timestamp = createDocument(dataStreamName);
DataStreamsStatsAction.Response stats = getDataStreamsStats(true);
@ -221,14 +250,19 @@ public class DataStreamsStatsTests extends ESSingleNodeTestCase {
}
private String createDataStream() throws Exception {
return createDataStream(false);
return createDataStream(false, false);
}
private String createDataStream(boolean hidden) throws Exception {
private String createDataStream(boolean hidden, boolean failureStore) throws Exception {
String dataStreamName = randomAlphaOfLength(10).toLowerCase(Locale.getDefault());
ResettableValue<DataStreamOptions.Template> failureStoreOptions = failureStore == false
? ResettableValue.undefined()
: ResettableValue.create(
new DataStreamOptions.Template(ResettableValue.create(new DataStreamFailureStore.Template(ResettableValue.create(true))))
);
Template idxTemplate = new Template(null, new CompressedXContent("""
{"properties":{"@timestamp":{"type":"date"},"data":{"type":"keyword"}}}
"""), null);
"""), null, null, failureStoreOptions);
ComposableIndexTemplate template = ComposableIndexTemplate.builder()
.indexPatterns(List.of(dataStreamName + "*"))
.template(idxTemplate)
@ -269,6 +303,27 @@ public class DataStreamsStatsTests extends ESSingleNodeTestCase {
return timestamp;
}
private long createFailedDocument(String dataStreamName) throws Exception {
// Get some randomized but reasonable timestamps on the data since not all of it is guaranteed to arrive in order.
long timeSeed = System.currentTimeMillis();
long timestamp = randomLongBetween(timeSeed - TimeUnit.HOURS.toMillis(5), timeSeed);
client().bulk(
new BulkRequest(dataStreamName).add(
new IndexRequest().opType(DocWriteRequest.OpType.CREATE)
.source(
JsonXContent.contentBuilder()
.startObject()
.field("@timestamp", timestamp)
.object("data", b -> b.field("garbage", randomAlphaOfLength(25)))
.endObject()
)
)
).get();
indicesAdmin().refresh(new RefreshRequest(".fs-" + dataStreamName + "*").indicesOptions(IndicesOptions.lenientExpandOpenHidden()))
.get();
return timestamp;
}
private DataStreamsStatsAction.Response getDataStreamsStats() throws Exception {
return getDataStreamsStats(false);
}

View file

@ -27,7 +27,7 @@ import org.elasticsearch.action.datastreams.lifecycle.ErrorEntry;
import org.elasticsearch.action.downsample.DownsampleAction;
import org.elasticsearch.action.downsample.DownsampleConfig;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterName;
@ -46,6 +46,7 @@ import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexGraveyard;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.MetadataIndexStateService;
import org.elasticsearch.cluster.node.DiscoveryNode;
@ -225,11 +226,12 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
assertThat(clientSeenRequests.get(0), instanceOf(RolloverRequest.class));
RolloverRequest rolloverBackingIndexRequest = (RolloverRequest) clientSeenRequests.get(0);
assertThat(rolloverBackingIndexRequest.getRolloverTarget(), is(dataStreamName));
assertThat(rolloverBackingIndexRequest.indicesOptions().selectorOptions(), equalTo(IndicesOptions.SelectorOptions.DATA));
assertThat(clientSeenRequests.get(1), instanceOf(RolloverRequest.class));
RolloverRequest rolloverFailureIndexRequest = (RolloverRequest) clientSeenRequests.get(1);
assertThat(rolloverFailureIndexRequest.getRolloverTarget(), is(dataStreamName));
assertThat(rolloverFailureIndexRequest.indicesOptions().selectorOptions(), equalTo(IndicesOptions.SelectorOptions.FAILURES));
assertThat(
rolloverFailureIndexRequest.getRolloverTarget(),
is(IndexNameExpressionResolver.combineSelector(dataStreamName, IndexComponentSelector.FAILURES))
);
List<DeleteIndexRequest> deleteRequests = clientSeenRequests.subList(2, 5)
.stream()
.map(transportRequest -> (DeleteIndexRequest) transportRequest)
@ -1546,11 +1548,12 @@ public class DataStreamLifecycleServiceTests extends ESTestCase {
assertThat(clientSeenRequests.get(0), instanceOf(RolloverRequest.class));
RolloverRequest rolloverBackingIndexRequest = (RolloverRequest) clientSeenRequests.get(0);
assertThat(rolloverBackingIndexRequest.getRolloverTarget(), is(dataStreamName));
assertThat(rolloverBackingIndexRequest.indicesOptions().selectorOptions(), equalTo(IndicesOptions.SelectorOptions.DATA));
assertThat(clientSeenRequests.get(1), instanceOf(RolloverRequest.class));
RolloverRequest rolloverFailureIndexRequest = (RolloverRequest) clientSeenRequests.get(1);
assertThat(rolloverFailureIndexRequest.getRolloverTarget(), is(dataStreamName));
assertThat(rolloverFailureIndexRequest.indicesOptions().selectorOptions(), equalTo(IndicesOptions.SelectorOptions.FAILURES));
assertThat(
rolloverFailureIndexRequest.getRolloverTarget(),
is(IndexNameExpressionResolver.combineSelector(dataStreamName, IndexComponentSelector.FAILURES))
);
assertThat(
((DeleteIndexRequest) clientSeenRequests.get(2)).indices()[0],
is(dataStream.getFailureIndices().getIndices().get(0).getName())

View file

@ -148,8 +148,7 @@
# rollover data stream to create new failure store index
- do:
indices.rollover:
alias: "data-stream-for-modification"
target_failure_store: true
alias: "data-stream-for-modification::failures"
- is_true: acknowledged
# save index names for later use

View file

@ -9,7 +9,7 @@ setup:
capabilities: [ 'failure_store_in_template' ]
- method: POST
path: /{index}/_rollover
capabilities: [ 'lazy-rollover-failure-store' ]
capabilities: [ 'lazy-rollover-failure-store', 'index-expression-selectors' ]
- do:
allowed_warnings:
@ -58,8 +58,7 @@ teardown:
- do:
indices.rollover:
alias: "data-stream-for-rollover"
target_failure_store: true
alias: "data-stream-for-rollover::failures"
- match: { acknowledged: true }
- match: { old_index: "/\\.fs-data-stream-for-rollover-(\\d{4}\\.\\d{2}\\.\\d{2}-)?000002/" }
@ -92,8 +91,7 @@ teardown:
- do:
indices.rollover:
alias: "data-stream-for-rollover"
target_failure_store: true
alias: "data-stream-for-rollover::failures"
body:
conditions:
max_docs: 1
@ -130,8 +128,7 @@ teardown:
- do:
indices.rollover:
alias: "data-stream-for-rollover"
target_failure_store: true
alias: "data-stream-for-rollover::failures"
body:
conditions:
max_primary_shard_docs: 2
@ -165,8 +162,7 @@ teardown:
# Mark the failure store for lazy rollover
- do:
indices.rollover:
alias: "data-stream-for-rollover"
target_failure_store: true
alias: "data-stream-for-rollover::failures"
lazy: true
- match: { acknowledged: true }
@ -263,8 +259,7 @@ teardown:
# Mark the failure store for lazy rollover
- do:
indices.rollover:
alias: data-stream-for-lazy-rollover
target_failure_store: true
alias: data-stream-for-lazy-rollover::failures
lazy: true
- match: { acknowledged: true }
@ -332,8 +327,7 @@ teardown:
# Mark the failure store for lazy rollover
- do:
indices.rollover:
alias: "data-stream-for-rollover"
target_failure_store: true
alias: "data-stream-for-rollover::failures"
lazy: true
- match: { acknowledged: true }
@ -377,16 +371,14 @@ teardown:
- do:
catch: /Rolling over\/initializing an empty failure store is only supported without conditions\./
indices.rollover:
alias: "data-stream-for-rollover"
target_failure_store: true
alias: "data-stream-for-rollover::failures"
body:
conditions:
max_docs: 1
- do:
indices.rollover:
alias: "data-stream-for-rollover"
target_failure_store: true
alias: "data-stream-for-rollover::failures"
- match: { acknowledged: true }
- match: { old_index: "_none_" }
@ -424,8 +416,7 @@ teardown:
# Initializing should work
- do:
indices.rollover:
alias: "other-data-stream-for-rollover"
target_failure_store: true
alias: "other-data-stream-for-rollover::failures"
- match: { acknowledged: true }
- match: { old_index: "_none_" }
@ -448,8 +439,7 @@ teardown:
# And "regular" rollover should work
- do:
indices.rollover:
alias: "other-data-stream-for-rollover"
target_failure_store: true
alias: "other-data-stream-for-rollover::failures"
- match: { acknowledged: true }
- match: { old_index: "/\\.fs-other-data-stream-for-rollover-(\\d{4}\\.\\d{2}\\.\\d{2}-)?000002/" }

View file

@ -63,12 +63,6 @@
"type":"boolean",
"default":"false",
"description":"If set to true, the rollover action will only mark a data stream to signal that it needs to be rolled over at the next write. Only allowed on data streams."
},
"target_failure_store":{
"type":"boolean",
"description":"If set to true, the rollover action will be applied on the failure store of the data stream.",
"visibility": "feature_flag",
"feature_flag": "es.failure_store_feature_flag_enabled"
}
},
"body":{

View file

@ -153,6 +153,7 @@ public class TransportVersions {
public static final TransportVersion ESQL_ENABLE_NODE_LEVEL_REDUCTION = def(8_818_00_0);
public static final TransportVersion JINA_AI_INTEGRATION_ADDED = def(8_819_00_0);
public static final TransportVersion TRACK_INDEX_FAILED_DUE_TO_VERSION_CONFLICT_METRIC = def(8_820_00_0);
public static final TransportVersion REPLACE_FAILURE_STORE_OPTIONS_WITH_SELECTOR_SYNTAX = def(8_821_00_0);
/*
* STOP! READ THIS FIRST! No, really,

View file

@ -14,7 +14,6 @@ import org.elasticsearch.TransportVersions;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.MasterNodeRequest;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -44,9 +43,7 @@ public class RestoreSnapshotRequest extends MasterNodeRequest<RestoreSnapshotReq
private String snapshot;
private String repository;
private String[] indices = Strings.EMPTY_ARRAY;
private IndicesOptions indicesOptions = DataStream.isFailureStoreFeatureFlagEnabled()
? IndicesOptions.strictExpandOpenIncludeFailureStore()
: IndicesOptions.strictExpandOpen();
private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpen();
private String[] featureStates = Strings.EMPTY_ARRAY;
private String renamePattern;
private String renameReplacement;

View file

@ -74,7 +74,7 @@ public class IndicesAliasesRequest extends AcknowledgedRequest<IndicesAliasesReq
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.ignoreThrottled(false)
.allowFailureIndices(true)
.allowSelectors(false)
.build()
)
.build();

View file

@ -23,7 +23,7 @@ import java.util.Map;
public class GetAliasesRequest extends LocalClusterStateRequest implements AliasesRequest {
public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.strictExpandHidden();
public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.strictExpandHiddenNoSelectors();
private String[] aliases;
private String[] originalAliases;

View file

@ -43,7 +43,7 @@ public class DeleteIndexRequest extends AcknowledgedRequest<DeleteIndexRequest>
.allowAliasToMultipleIndices(false)
.allowClosedIndices(true)
.ignoreThrottled(false)
.allowFailureIndices(true)
.allowSelectors(false)
.build()
)
.build();

View file

@ -12,7 +12,6 @@ package org.elasticsearch.action.admin.indices.get;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.info.ClusterInfoRequest;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.util.ArrayUtils;
@ -95,13 +94,7 @@ public class GetIndexRequest extends ClusterInfoRequest<GetIndexRequest> {
private transient boolean includeDefaults = false;
public GetIndexRequest() {
super(
DataStream.isFailureStoreFeatureFlagEnabled()
? IndicesOptions.builder(IndicesOptions.strictExpandOpen())
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.build()
: IndicesOptions.strictExpandOpen()
);
super(IndicesOptions.strictExpandOpen());
}
public GetIndexRequest(StreamInput in) throws IOException {

View file

@ -82,7 +82,7 @@ public class PutMappingRequest extends AcknowledgedRequest<PutMappingRequest> im
.allowClosedIndices(true)
.allowAliasToMultipleIndices(true)
.ignoreThrottled(false)
.allowFailureIndices(false)
.allowSelectors(false)
)
.build();

View file

@ -20,6 +20,8 @@ import org.elasticsearch.action.support.master.AcknowledgedTransportMasterNodeAc
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetadataMappingService;
import org.elasticsearch.cluster.service.ClusterService;
@ -40,6 +42,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.SortedMap;
/**
* Put mapping action.
@ -106,7 +109,14 @@ public class TransportPutMappingAction extends AcknowledgedTransportMasterNodeAc
return;
}
final String message = checkForSystemIndexViolations(systemIndices, concreteIndices, request);
String message = checkForFailureStoreViolations(clusterService.state(), concreteIndices, request);
if (message != null) {
logger.warn(message);
listener.onFailure(new IllegalStateException(message));
return;
}
message = checkForSystemIndexViolations(systemIndices, concreteIndices, request);
if (message != null) {
logger.warn(message);
listener.onFailure(new IllegalStateException(message));
@ -172,6 +182,33 @@ public class TransportPutMappingAction extends AcknowledgedTransportMasterNodeAc
metadataMappingService.putMapping(updateRequest, wrappedListener);
}
static String checkForFailureStoreViolations(ClusterState clusterState, Index[] concreteIndices, PutMappingRequest request) {
// Requests that a cluster generates itself are permitted to make changes to mappings
// so that rolling upgrade scenarios still work. We check this via the request's origin.
if (Strings.isNullOrEmpty(request.origin()) == false) {
return null;
}
List<String> violations = new ArrayList<>();
SortedMap<String, IndexAbstraction> indicesLookup = clusterState.metadata().getIndicesLookup();
for (Index index : concreteIndices) {
IndexAbstraction indexAbstraction = indicesLookup.get(index.getName());
if (indexAbstraction != null) {
DataStream maybeDataStream = indexAbstraction.getParentDataStream();
if (maybeDataStream != null && maybeDataStream.isFailureStoreIndex(index.getName())) {
violations.add(index.getName());
}
}
}
if (violations.isEmpty() == false) {
return "Cannot update mappings in "
+ violations
+ ": mappings for indices contained in data stream failure stores cannot be updated";
}
return null;
}
static String checkForSystemIndexViolations(SystemIndices systemIndices, Index[] concreteIndices, PutMappingRequest request) {
// Requests that a cluster generates itself are permitted to have a difference in mappings
// so that rolling upgrade scenarios still work. We check this via the request's origin.

View file

@ -59,6 +59,7 @@ import java.util.Objects;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.stream.Stream;
import static org.elasticsearch.action.search.TransportSearchHelper.checkCCSVersionCompatibility;
@ -598,12 +599,13 @@ public class ResolveIndexAction extends ActionType<ResolveIndexAction.Response>
private static void enrichIndexAbstraction(
ClusterState clusterState,
ResolvedExpression indexAbstraction,
ResolvedExpression resolvedExpression,
List<ResolvedIndex> indices,
List<ResolvedAlias> aliases,
List<ResolvedDataStream> dataStreams
) {
IndexAbstraction ia = clusterState.metadata().getIndicesLookup().get(indexAbstraction.resource());
SortedMap<String, IndexAbstraction> indicesLookup = clusterState.metadata().getIndicesLookup();
IndexAbstraction ia = indicesLookup.get(resolvedExpression.resource());
if (ia != null) {
switch (ia.getType()) {
case CONCRETE_INDEX -> {
@ -632,13 +634,24 @@ public class ResolveIndexAction extends ActionType<ResolveIndexAction.Response>
);
}
case ALIAS -> {
String[] indexNames = ia.getIndices().stream().map(Index::getName).toArray(String[]::new);
String[] indexNames = getAliasIndexStream(resolvedExpression, ia, indicesLookup).map(Index::getName)
.toArray(String[]::new);
Arrays.sort(indexNames);
aliases.add(new ResolvedAlias(ia.getName(), indexNames));
}
case DATA_STREAM -> {
DataStream dataStream = (DataStream) ia;
String[] backingIndices = dataStream.getIndices().stream().map(Index::getName).toArray(String[]::new);
Stream<Index> dataStreamIndices = resolvedExpression.selector() == null
? dataStream.getIndices().stream()
: switch (resolvedExpression.selector()) {
case DATA -> dataStream.getBackingIndices().getIndices().stream();
case FAILURES -> dataStream.getFailureIndices().getIndices().stream();
case ALL_APPLICABLE -> Stream.concat(
dataStream.getBackingIndices().getIndices().stream(),
dataStream.getFailureIndices().getIndices().stream()
);
};
String[] backingIndices = dataStreamIndices.map(Index::getName).toArray(String[]::new);
dataStreams.add(new ResolvedDataStream(dataStream.getName(), backingIndices, DataStream.TIMESTAMP_FIELD_NAME));
}
default -> throw new IllegalStateException("unknown index abstraction type: " + ia.getType());
@ -646,6 +659,52 @@ public class ResolveIndexAction extends ActionType<ResolveIndexAction.Response>
}
}
private static Stream<Index> getAliasIndexStream(
ResolvedExpression resolvedExpression,
IndexAbstraction ia,
SortedMap<String, IndexAbstraction> indicesLookup
) {
Stream<Index> aliasIndices;
if (resolvedExpression.selector() == null) {
aliasIndices = ia.getIndices().stream();
} else {
aliasIndices = switch (resolvedExpression.selector()) {
case DATA -> ia.getIndices().stream();
case FAILURES -> {
assert ia.isDataStreamRelated() : "Illegal selector [failures] used on non data stream alias";
yield ia.getIndices()
.stream()
.map(Index::getName)
.map(indicesLookup::get)
.map(IndexAbstraction::getParentDataStream)
.filter(Objects::nonNull)
.distinct()
.map(DataStream::getFailureIndices)
.flatMap(failureIndices -> failureIndices.getIndices().stream());
}
case ALL_APPLICABLE -> {
if (ia.isDataStreamRelated()) {
yield Stream.concat(
ia.getIndices().stream(),
ia.getIndices()
.stream()
.map(Index::getName)
.map(indicesLookup::get)
.map(IndexAbstraction::getParentDataStream)
.filter(Objects::nonNull)
.distinct()
.map(DataStream::getFailureIndices)
.flatMap(failureIndices -> failureIndices.getIndices().stream())
);
} else {
yield ia.getIndices().stream();
}
}
};
}
return aliasIndices;
}
enum Attribute {
OPEN,
CLOSED,

View file

@ -21,6 +21,8 @@ import org.elasticsearch.cluster.ClusterStateTaskExecutor;
import org.elasticsearch.cluster.ClusterStateTaskListener;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SelectorResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.MetadataCreateIndexService;
import org.elasticsearch.cluster.metadata.MetadataDataStreamsService;
@ -119,32 +121,38 @@ public final class LazyRolloverAction extends ActionType<RolloverResponse> {
: "The auto rollover action does not expect any other parameters in the request apart from the data stream name";
Metadata metadata = clusterState.metadata();
DataStream dataStream = metadata.dataStreams().get(rolloverRequest.getRolloverTarget());
ResolvedExpression resolvedRolloverTarget = SelectorResolver.parseExpression(
rolloverRequest.getRolloverTarget(),
rolloverRequest.indicesOptions()
);
boolean isFailureStoreRollover = resolvedRolloverTarget.selector() != null
&& resolvedRolloverTarget.selector().shouldIncludeFailures();
DataStream dataStream = metadata.dataStreams().get(resolvedRolloverTarget.resource());
// Skip submitting the task if we detect that the lazy rollover has been already executed.
if (isLazyRolloverNeeded(dataStream, rolloverRequest.targetsFailureStore()) == false) {
DataStream.DataStreamIndices targetIndices = dataStream.getDataStreamIndices(rolloverRequest.targetsFailureStore());
if (isLazyRolloverNeeded(dataStream, isFailureStoreRollover) == false) {
DataStream.DataStreamIndices targetIndices = dataStream.getDataStreamIndices(isFailureStoreRollover);
listener.onResponse(noopLazyRolloverResponse(targetIndices));
return;
}
// We evaluate the names of the source index as well as what our newly created index would be.
final MetadataRolloverService.NameResolution trialRolloverNames = MetadataRolloverService.resolveRolloverNames(
clusterState,
rolloverRequest.getRolloverTarget(),
resolvedRolloverTarget.resource(),
rolloverRequest.getNewIndexName(),
rolloverRequest.getCreateIndexRequest(),
rolloverRequest.targetsFailureStore()
isFailureStoreRollover
);
final String trialSourceIndexName = trialRolloverNames.sourceName();
final String trialRolloverIndexName = trialRolloverNames.rolloverName();
MetadataCreateIndexService.validateIndexName(trialRolloverIndexName, clusterState.metadata(), clusterState.routingTable());
assert metadata.dataStreams().containsKey(rolloverRequest.getRolloverTarget()) : "Auto-rollover applies only to data streams";
assert metadata.dataStreams().containsKey(resolvedRolloverTarget.resource()) : "Auto-rollover applies only to data streams";
String source = "lazy_rollover source [" + trialSourceIndexName + "] to target [" + trialRolloverIndexName + "]";
// We create a new rollover request to ensure that it doesn't contain any other parameters apart from the data stream name
// This will provide a more resilient user experience
var newRolloverRequest = new RolloverRequest(rolloverRequest.getRolloverTarget(), null);
newRolloverRequest.setIndicesOptions(rolloverRequest.indicesOptions());
var newRolloverRequest = new RolloverRequest(resolvedRolloverTarget.combined(), null);
LazyRolloverTask rolloverTask = new LazyRolloverTask(newRolloverRequest, listener);
lazyRolloverTaskQueue.submitTask(source, rolloverTask, rolloverRequest.masterNodeTimeout());
}
@ -223,12 +231,19 @@ public final class LazyRolloverAction extends ActionType<RolloverResponse> {
AllocationActionMultiListener<RolloverResponse> allocationActionMultiListener
) throws Exception {
ResolvedExpression resolvedRolloverTarget = SelectorResolver.parseExpression(
rolloverRequest.getRolloverTarget(),
rolloverRequest.indicesOptions()
);
boolean isFailureStoreRollover = resolvedRolloverTarget.selector() != null
&& resolvedRolloverTarget.selector().shouldIncludeFailures();
// If the data stream has been rolled over since it was marked for lazy rollover, this operation is a noop
final DataStream dataStream = currentState.metadata().dataStreams().get(rolloverRequest.getRolloverTarget());
final DataStream dataStream = currentState.metadata().dataStreams().get(resolvedRolloverTarget.resource());
assert dataStream != null;
if (isLazyRolloverNeeded(dataStream, rolloverRequest.targetsFailureStore()) == false) {
final DataStream.DataStreamIndices targetIndices = dataStream.getDataStreamIndices(rolloverRequest.targetsFailureStore());
if (isLazyRolloverNeeded(dataStream, isFailureStoreRollover) == false) {
final DataStream.DataStreamIndices targetIndices = dataStream.getDataStreamIndices(isFailureStoreRollover);
var noopResponse = noopLazyRolloverResponse(targetIndices);
notifyAllListeners(rolloverTaskContexts, context -> context.getTask().listener.onResponse(noopResponse));
return currentState;
@ -237,7 +252,7 @@ public final class LazyRolloverAction extends ActionType<RolloverResponse> {
// Perform the actual rollover
final var rolloverResult = rolloverService.rolloverClusterState(
currentState,
rolloverRequest.getRolloverTarget(),
resolvedRolloverTarget.resource(),
rolloverRequest.getNewIndexName(),
rolloverRequest.getCreateIndexRequest(),
List.of(),
@ -246,7 +261,7 @@ public final class LazyRolloverAction extends ActionType<RolloverResponse> {
false,
null,
null,
rolloverRequest.targetsFailureStore()
isFailureStoreRollover
);
results.add(rolloverResult);
logger.trace("lazy rollover result [{}]", rolloverResult);

View file

@ -16,7 +16,8 @@ import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.AcknowledgedRequest;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SelectorResolver;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.mapper.MapperService;
@ -81,7 +82,7 @@ public class RolloverRequest extends AcknowledgedRequest<RolloverRequest> implem
private RolloverConditions conditions = new RolloverConditions();
// the index name "_na_" is never read back, what matters are settings, mappings and aliases
private CreateIndexRequest createIndexRequest = new CreateIndexRequest("_na_");
private IndicesOptions indicesOptions = IndicesOptions.strictSingleIndexNoExpandForbidClosed();
private IndicesOptions indicesOptions = IndicesOptions.strictSingleIndexNoExpandForbidClosedAllowSelectors();
public RolloverRequest(StreamInput in) throws IOException {
super(in);
@ -125,13 +126,16 @@ public class RolloverRequest extends AcknowledgedRequest<RolloverRequest> implem
);
}
var selector = indicesOptions.selectorOptions().defaultSelector();
if (selector == IndexComponentSelector.ALL_APPLICABLE) {
if (rolloverTarget != null) {
ResolvedExpression resolvedExpression = SelectorResolver.parseExpression(rolloverTarget, indicesOptions);
IndexComponentSelector selector = resolvedExpression.selector();
if (IndexComponentSelector.ALL_APPLICABLE.equals(selector)) {
validationException = addValidationError(
"rollover cannot be applied to both regular and failure indices at the same time",
validationException
);
}
}
return validationException;
}
@ -162,13 +166,6 @@ public class RolloverRequest extends AcknowledgedRequest<RolloverRequest> implem
return indicesOptions;
}
/**
* @return true of the rollover request targets the failure store, false otherwise.
*/
public boolean targetsFailureStore() {
return DataStream.isFailureStoreFeatureFlagEnabled() && indicesOptions.includeFailureIndices();
}
public void setIndicesOptions(IndicesOptions indicesOptions) {
this.indicesOptions = indicesOptions;
}

View file

@ -36,6 +36,8 @@ import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexMetadataStats;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SelectorResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.MetadataCreateIndexService;
import org.elasticsearch.cluster.metadata.MetadataDataStreamsService;
@ -149,8 +151,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
.matchOpen(request.indicesOptions().expandWildcardsOpen())
.matchClosed(request.indicesOptions().expandWildcardsClosed())
.build(),
IndicesOptions.GatekeeperOptions.DEFAULT,
request.indicesOptions().selectorOptions()
IndicesOptions.GatekeeperOptions.DEFAULT
);
return state.blocks()
@ -170,11 +171,18 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
assert task instanceof CancellableTask;
Metadata metadata = clusterState.metadata();
// Parse the rollover request's target since the expression it may contain a selector on it
ResolvedExpression resolvedRolloverTarget = SelectorResolver.parseExpression(
rolloverRequest.getRolloverTarget(),
rolloverRequest.indicesOptions()
);
boolean targetFailureStore = resolvedRolloverTarget.selector() != null && resolvedRolloverTarget.selector().shouldIncludeFailures();
// We evaluate the names of the index for which we should evaluate conditions, as well as what our newly created index *would* be.
boolean targetFailureStore = rolloverRequest.targetsFailureStore();
final MetadataRolloverService.NameResolution trialRolloverNames = MetadataRolloverService.resolveRolloverNames(
clusterState,
rolloverRequest.getRolloverTarget(),
resolvedRolloverTarget.resource(),
rolloverRequest.getNewIndexName(),
rolloverRequest.getCreateIndexRequest(),
targetFailureStore
@ -183,7 +191,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
final String trialRolloverIndexName = trialRolloverNames.rolloverName();
MetadataCreateIndexService.validateIndexName(trialRolloverIndexName, metadata, clusterState.routingTable());
boolean isDataStream = metadata.dataStreams().containsKey(rolloverRequest.getRolloverTarget());
boolean isDataStream = metadata.dataStreams().containsKey(resolvedRolloverTarget.resource());
if (rolloverRequest.isLazy()) {
if (isDataStream == false || rolloverRequest.getConditions().hasConditions()) {
String message;
@ -201,7 +209,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
}
if (rolloverRequest.isDryRun() == false) {
metadataDataStreamsService.setRolloverOnWrite(
rolloverRequest.getRolloverTarget(),
resolvedRolloverTarget.resource(),
true,
targetFailureStore,
rolloverRequest.ackTimeout(),
@ -225,7 +233,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
final IndexAbstraction rolloverTargetAbstraction = clusterState.metadata()
.getIndicesLookup()
.get(rolloverRequest.getRolloverTarget());
.get(resolvedRolloverTarget.resource());
if (rolloverTargetAbstraction.getType() == IndexAbstraction.Type.ALIAS && rolloverTargetAbstraction.isDataStreamRelated()) {
listener.onFailure(
new IllegalStateException("Aliases to data streams cannot be rolled over. Please rollover the data stream itself.")
@ -246,10 +254,10 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
final var statsIndicesOptions = new IndicesOptions(
IndicesOptions.ConcreteTargetOptions.ALLOW_UNAVAILABLE_TARGETS,
IndicesOptions.WildcardOptions.builder().matchClosed(true).allowEmptyExpressions(false).build(),
IndicesOptions.GatekeeperOptions.DEFAULT,
rolloverRequest.indicesOptions().selectorOptions()
IndicesOptions.GatekeeperOptions.DEFAULT
);
IndicesStatsRequest statsRequest = new IndicesStatsRequest().indices(rolloverRequest.getRolloverTarget())
// Make sure to recombine any selectors on the stats request
IndicesStatsRequest statsRequest = new IndicesStatsRequest().indices(resolvedRolloverTarget.combined())
.clear()
.indicesOptions(statsIndicesOptions)
.docs(true)
@ -266,9 +274,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
listener.delegateFailureAndWrap((delegate, statsResponse) -> {
AutoShardingResult rolloverAutoSharding = null;
final IndexAbstraction indexAbstraction = clusterState.metadata()
.getIndicesLookup()
.get(rolloverRequest.getRolloverTarget());
final IndexAbstraction indexAbstraction = clusterState.metadata().getIndicesLookup().get(resolvedRolloverTarget.resource());
if (indexAbstraction.getType().equals(IndexAbstraction.Type.DATA_STREAM)) {
DataStream dataStream = (DataStream) indexAbstraction;
final Optional<IndexStats> indexStats = Optional.ofNullable(statsResponse)
@ -492,14 +498,20 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
) throws Exception {
final var rolloverTask = rolloverTaskContext.getTask();
final var rolloverRequest = rolloverTask.rolloverRequest();
ResolvedExpression resolvedRolloverTarget = SelectorResolver.parseExpression(
rolloverRequest.getRolloverTarget(),
rolloverRequest.indicesOptions()
);
boolean targetFailureStore = resolvedRolloverTarget.selector() != null
&& resolvedRolloverTarget.selector().shouldIncludeFailures();
// Regenerate the rollover names, as a rollover could have happened in between the pre-check and the cluster state update
final var rolloverNames = MetadataRolloverService.resolveRolloverNames(
currentState,
rolloverRequest.getRolloverTarget(),
resolvedRolloverTarget.resource(),
rolloverRequest.getNewIndexName(),
rolloverRequest.getCreateIndexRequest(),
rolloverRequest.targetsFailureStore()
targetFailureStore
);
// Re-evaluate the conditions, now with our final source index name
@ -532,7 +544,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
final IndexAbstraction rolloverTargetAbstraction = currentState.metadata()
.getIndicesLookup()
.get(rolloverRequest.getRolloverTarget());
.get(resolvedRolloverTarget.resource());
final IndexMetadataStats sourceIndexStats = rolloverTargetAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM
? IndexMetadataStats.fromStatsResponse(rolloverSourceIndex, rolloverTask.statsResponse())
@ -541,7 +553,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
// Perform the actual rollover
final var rolloverResult = rolloverService.rolloverClusterState(
currentState,
rolloverRequest.getRolloverTarget(),
resolvedRolloverTarget.resource(),
rolloverRequest.getNewIndexName(),
rolloverRequest.getCreateIndexRequest(),
metConditions,
@ -550,7 +562,7 @@ public class TransportRolloverAction extends TransportMasterNodeAction<RolloverR
false,
sourceIndexStats,
rolloverTask.autoShardingResult(),
rolloverRequest.targetsFailureStore()
targetFailureStore
);
results.add(rolloverResult);
logger.trace("rollover result [{}]", rolloverResult);

View file

@ -24,7 +24,7 @@ import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.RefCountingRunnable;
import org.elasticsearch.client.internal.OriginSettingClient;
import org.elasticsearch.client.internal.node.NodeClient;
@ -216,11 +216,9 @@ final class BulkOperation extends ActionRunnable<BulkResponse> {
}
try (RefCountingRunnable refs = new RefCountingRunnable(runnable)) {
for (String dataStream : failureStoresToBeRolledOver) {
RolloverRequest rolloverRequest = new RolloverRequest(dataStream, null);
rolloverRequest.setIndicesOptions(
IndicesOptions.builder(rolloverRequest.indicesOptions())
.selectorOptions(IndicesOptions.SelectorOptions.FAILURES)
.build()
RolloverRequest rolloverRequest = new RolloverRequest(
IndexNameExpressionResolver.combineSelector(dataStream, IndexComponentSelector.FAILURES),
null
);
// We are executing a lazy rollover because it is an action specialised for this situation, when we want an
// unconditional and performant rollover.

View file

@ -25,7 +25,7 @@ import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.RefCountingRunnable;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.action.support.replication.ReplicationResponse;
@ -425,11 +425,7 @@ public class TransportBulkAction extends TransportAbstractBulkAction {
RolloverRequest rolloverRequest = new RolloverRequest(dataStream, null);
rolloverRequest.masterNodeTimeout(bulkRequest.timeout);
if (targetFailureStore) {
rolloverRequest.setIndicesOptions(
IndicesOptions.builder(rolloverRequest.indicesOptions())
.selectorOptions(IndicesOptions.SelectorOptions.FAILURES)
.build()
);
rolloverRequest.setRolloverTarget(IndexNameExpressionResolver.combineSelector(dataStream, IndexComponentSelector.FAILURES));
}
// We are executing a lazy rollover because it is an action specialised for this situation, when we want an
// unconditional and performant rollover.
@ -438,9 +434,8 @@ public class TransportBulkAction extends TransportAbstractBulkAction {
@Override
public void onResponse(RolloverResponse result) {
logger.debug(
"Data stream{} {} has {} over, the latest index is {}",
rolloverRequest.targetsFailureStore() ? " failure store" : "",
dataStream,
"Data stream [{}] has {} over, the latest index is {}",
rolloverRequest.getRolloverTarget(),
result.isRolledOver() ? "been successfully rolled" : "skipped rolling",
result.getNewIndex()
);

View file

@ -9,16 +9,18 @@
package org.elasticsearch.action.datastreams;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
import org.elasticsearch.index.Index;
import java.util.ArrayList;
import java.util.List;
import java.util.SortedMap;
import java.util.stream.Stream;
public class DataStreamsActionUtil {
@ -47,25 +49,79 @@ public class DataStreamsActionUtil {
return indicesOptions;
}
public static Stream<String> resolveConcreteIndexNames(
public static List<String> resolveConcreteIndexNames(
IndexNameExpressionResolver indexNameExpressionResolver,
ClusterState clusterState,
String[] names,
IndicesOptions indicesOptions
) {
List<String> abstractionNames = getDataStreamNames(indexNameExpressionResolver, clusterState, names, indicesOptions);
List<ResolvedExpression> abstractionNames = indexNameExpressionResolver.dataStreams(
clusterState,
updateIndicesOptions(indicesOptions),
names
);
SortedMap<String, IndexAbstraction> indicesLookup = clusterState.getMetadata().getIndicesLookup();
return abstractionNames.stream().flatMap(abstractionName -> {
List<String> results = new ArrayList<>(abstractionNames.size());
for (ResolvedExpression abstractionName : abstractionNames) {
IndexAbstraction indexAbstraction = indicesLookup.get(abstractionName.resource());
assert indexAbstraction != null;
if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM) {
selectDataStreamIndicesNames(
(DataStream) indexAbstraction,
IndexComponentSelector.FAILURES.equals(abstractionName.selector()),
results
);
}
}
return results;
}
/**
* Resolves a list of expressions into data stream names and then collects the concrete indices
* that are applicable for those data streams based on the selector provided in the arguments.
* @param indexNameExpressionResolver resolver object
* @param clusterState state to query
* @param names data stream expressions
* @param selector which component indices of the data stream should be returned
* @param indicesOptions options for expression resolution
* @return A stream of concrete index names that belong to the components specified
* on the data streams returned from the expressions given
*/
public static List<String> resolveConcreteIndexNamesWithSelector(
IndexNameExpressionResolver indexNameExpressionResolver,
ClusterState clusterState,
String[] names,
IndexComponentSelector selector,
IndicesOptions indicesOptions
) {
assert indicesOptions.allowSelectors() == false : "If selectors are enabled, use resolveConcreteIndexNames instead";
List<String> abstractionNames = indexNameExpressionResolver.dataStreamNames(
clusterState,
updateIndicesOptions(indicesOptions),
names
);
SortedMap<String, IndexAbstraction> indicesLookup = clusterState.getMetadata().getIndicesLookup();
List<String> results = new ArrayList<>(abstractionNames.size());
for (String abstractionName : abstractionNames) {
IndexAbstraction indexAbstraction = indicesLookup.get(abstractionName);
assert indexAbstraction != null;
if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM) {
DataStream dataStream = (DataStream) indexAbstraction;
List<Index> indices = dataStream.getIndices();
return indices.stream().map(Index::getName);
} else {
return Stream.empty();
if (selector.shouldIncludeData()) {
selectDataStreamIndicesNames((DataStream) indexAbstraction, false, results);
}
if (selector.shouldIncludeFailures()) {
selectDataStreamIndicesNames((DataStream) indexAbstraction, true, results);
}
}
}
return results;
}
private static void selectDataStreamIndicesNames(DataStream indexAbstraction, boolean failureStore, List<String> accumulator) {
for (Index index : indexAbstraction.getDataStreamIndices(failureStore).getIndices()) {
accumulator.add(index.getName());
}
});
}
}

View file

@ -38,8 +38,6 @@ public class DataStreamsStatsAction extends ActionType<DataStreamsStatsAction.Re
public static class Request extends BroadcastRequest<Request> {
public Request() {
// this doesn't really matter since data stream name resolution isn't affected by IndicesOptions and
// a data stream's backing indices are retrieved from its metadata
super(
null,
IndicesOptions.builder()
@ -58,10 +56,9 @@ public class DataStreamsStatsAction extends ActionType<DataStreamsStatsAction.Re
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.ignoreThrottled(false)
.allowFailureIndices(true)
.allowSelectors(false)
.build()
)
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.build()
);
}

View file

@ -61,7 +61,7 @@ public class DeleteDataStreamAction extends ActionType<AcknowledgedResponse> {
.allowAliasToMultipleIndices(false)
.allowClosedIndices(true)
.ignoreThrottled(false)
.allowFailureIndices(true)
.allowSelectors(false)
.build()
)
.build();

View file

@ -72,10 +72,11 @@ public class GetDataStreamAction extends ActionType<GetDataStreamAction.Response
.allowAliasToMultipleIndices(false)
.allowClosedIndices(true)
.ignoreThrottled(false)
.allowFailureIndices(true)
.allowSelectors(false)
.build()
)
.build();
private boolean includeDefaults = false;
private boolean verbose = false;

View file

@ -63,7 +63,7 @@ public class GetDataStreamLifecycleAction {
.allowAliasToMultipleIndices(false)
.allowClosedIndices(true)
.ignoreThrottled(false)
.allowFailureIndices(true)
.allowSelectors(false)
.build()
)
.build();

View file

@ -94,7 +94,7 @@ public class PutDataStreamLifecycleAction {
.allowAliasToMultipleIndices(false)
.allowClosedIndices(true)
.ignoreThrottled(false)
.allowFailureIndices(false)
.allowSelectors(false)
.build()
)
.build();

View file

@ -82,7 +82,7 @@ public class DownsampleAction extends ActionType<AcknowledgedResponse> {
@Override
public IndicesOptions indicesOptions() {
return IndicesOptions.STRICT_SINGLE_INDEX_NO_EXPAND_FORBID_CLOSED;
return IndicesOptions.strictSingleIndexNoExpandForbidClosed();
}
@Override

View file

@ -13,7 +13,6 @@ import org.elasticsearch.TransportVersions;
import org.elasticsearch.cluster.metadata.DataStream;
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.logging.DeprecationCategory;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.core.Nullable;
@ -47,37 +46,13 @@ import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeSt
* @param gatekeeperOptions, applies to all the resolved indices and defines if throttled will be included and if certain type of
* aliases or indices are allowed, or they will throw an error. It acts as a gatekeeper when an action
* does not support certain options.
* @param selectorOptions, applies to all resolved expressions, and it specifies the index component that should be included, if there
* is no index component defined on the expression level.
*/
public record IndicesOptions(
ConcreteTargetOptions concreteTargetOptions,
WildcardOptions wildcardOptions,
GatekeeperOptions gatekeeperOptions,
SelectorOptions selectorOptions
GatekeeperOptions gatekeeperOptions
) implements ToXContentFragment {
/**
* @deprecated this query param will be replaced by the selector `::` on the expression level
*/
@Deprecated
public static final String FAILURE_STORE_QUERY_PARAM = "failure_store";
/**
* @deprecated this value will be replaced by the selector `::*` on the expression level
*/
@Deprecated
public static final String INCLUDE_ALL = "include";
/**
* @deprecated this value will be replaced by the selector `::data` on the expression level
*/
@Deprecated
public static final String INCLUDE_ONLY_REGULAR_INDICES = "exclude";
/**
* @deprecated this value will be replaced by the selector `::failures` on the expression level
*/
@Deprecated
public static final String INCLUDE_ONLY_FAILURE_INDICES = "only";
public static IndicesOptions.Builder builder() {
return new Builder();
}
@ -324,14 +299,14 @@ public record IndicesOptions(
* - The ignoreThrottled flag, which is a deprecated flag that will filter out frozen indices.
* @param allowAliasToMultipleIndices, allow aliases to multiple indices, true by default.
* @param allowClosedIndices, allow closed indices, true by default.
* @param allowFailureIndices, allow failure indices in the response, true by default
* @param allowSelectors, allow selectors within index expressions, true by default.
* @param ignoreThrottled, filters out throttled (aka frozen indices), defaults to true. This is deprecated and the only one
* that only filters and never throws an error.
*/
public record GatekeeperOptions(
boolean allowAliasToMultipleIndices,
boolean allowClosedIndices,
boolean allowFailureIndices,
boolean allowSelectors,
@Deprecated boolean ignoreThrottled
) implements ToXContentFragment {
@ -355,7 +330,7 @@ public record IndicesOptions(
public static class Builder {
private boolean allowAliasToMultipleIndices;
private boolean allowClosedIndices;
private boolean allowFailureIndices;
private boolean allowSelectors;
private boolean ignoreThrottled;
public Builder() {
@ -365,7 +340,7 @@ public record IndicesOptions(
Builder(GatekeeperOptions options) {
allowAliasToMultipleIndices = options.allowAliasToMultipleIndices;
allowClosedIndices = options.allowClosedIndices;
allowFailureIndices = options.allowFailureIndices;
allowSelectors = options.allowSelectors;
ignoreThrottled = options.ignoreThrottled;
}
@ -388,11 +363,12 @@ public record IndicesOptions(
}
/**
* Failure indices are accepted when true, otherwise the resolution will throw an error.
* Selectors are allowed within index expressions when true, otherwise the resolution will treat their presence as a syntax
* error when resolving index expressions.
* Defaults to true.
*/
public Builder allowFailureIndices(boolean allowFailureIndices) {
this.allowFailureIndices = allowFailureIndices;
public Builder allowSelectors(boolean allowSelectors) {
this.allowSelectors = allowSelectors;
return this;
}
@ -405,7 +381,7 @@ public record IndicesOptions(
}
public GatekeeperOptions build() {
return new GatekeeperOptions(allowAliasToMultipleIndices, allowClosedIndices, allowFailureIndices, ignoreThrottled);
return new GatekeeperOptions(allowAliasToMultipleIndices, allowClosedIndices, allowSelectors, ignoreThrottled);
}
}
@ -418,50 +394,6 @@ public record IndicesOptions(
}
}
/**
* Defines which selectors should be used by default for an index operation in the event that no selectors are provided.
*/
public record SelectorOptions(IndexComponentSelector defaultSelector) implements Writeable {
public static final SelectorOptions ALL_APPLICABLE = new SelectorOptions(IndexComponentSelector.ALL_APPLICABLE);
public static final SelectorOptions DATA = new SelectorOptions(IndexComponentSelector.DATA);
public static final SelectorOptions FAILURES = new SelectorOptions(IndexComponentSelector.FAILURES);
/**
* Default instance. Uses <pre>::data</pre> as the default selector if none are present in an index expression.
*/
public static final SelectorOptions DEFAULT = DATA;
public static SelectorOptions read(StreamInput in) throws IOException {
if (in.getTransportVersion().before(TransportVersions.INTRODUCE_ALL_APPLICABLE_SELECTOR)) {
EnumSet<IndexComponentSelector> set = in.readEnumSet(IndexComponentSelector.class);
if (set.isEmpty() || set.size() == 2) {
assert set.contains(IndexComponentSelector.DATA) && set.contains(IndexComponentSelector.FAILURES)
: "The enum set only supported ::data and ::failures";
return SelectorOptions.ALL_APPLICABLE;
} else if (set.contains(IndexComponentSelector.DATA)) {
return SelectorOptions.DATA;
} else {
return SelectorOptions.FAILURES;
}
} else {
return new SelectorOptions(IndexComponentSelector.read(in));
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
if (out.getTransportVersion().before(TransportVersions.INTRODUCE_ALL_APPLICABLE_SELECTOR)) {
switch (defaultSelector) {
case ALL_APPLICABLE -> out.writeEnumSet(EnumSet.of(IndexComponentSelector.DATA, IndexComponentSelector.FAILURES));
case DATA -> out.writeEnumSet(EnumSet.of(IndexComponentSelector.DATA));
case FAILURES -> out.writeEnumSet(EnumSet.of(IndexComponentSelector.FAILURES));
}
} else {
defaultSelector.writeTo(out);
}
}
}
/**
* This class is maintained for backwards compatibility and performance purposes. We use it for serialisation along with {@link Option}.
*/
@ -497,7 +429,8 @@ public record IndicesOptions(
ERROR_WHEN_CLOSED_INDICES,
IGNORE_THROTTLED,
ALLOW_FAILURE_INDICES // Added in 8.14
ALLOW_FAILURE_INDICES, // Added in 8.14, Removed in 8.18
ALLOW_SELECTORS // Added in 8.18
}
private static final DeprecationLogger DEPRECATION_LOGGER = DeprecationLogger.getLogger(IndicesOptions.class);
@ -510,8 +443,7 @@ public record IndicesOptions(
public static final IndicesOptions DEFAULT = new IndicesOptions(
ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS,
WildcardOptions.DEFAULT,
GatekeeperOptions.DEFAULT,
SelectorOptions.DEFAULT
GatekeeperOptions.DEFAULT
);
public static final IndicesOptions STRICT_EXPAND_OPEN = IndicesOptions.builder()
@ -528,10 +460,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions STRICT_EXPAND_OPEN_FAILURE_STORE = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -547,10 +478,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.ALL_APPLICABLE)
.build();
public static final IndicesOptions LENIENT_EXPAND_OPEN = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ALLOW_UNAVAILABLE_TARGETS)
@ -566,10 +496,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions LENIENT_EXPAND_OPEN_NO_SELECTORS = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ALLOW_UNAVAILABLE_TARGETS)
@ -585,7 +514,7 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(false)
.allowSelectors(false)
.ignoreThrottled(false)
)
.build();
@ -603,10 +532,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions LENIENT_EXPAND_OPEN_CLOSED = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ALLOW_UNAVAILABLE_TARGETS)
@ -622,10 +550,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions LENIENT_EXPAND_OPEN_CLOSED_HIDDEN = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ALLOW_UNAVAILABLE_TARGETS)
@ -636,10 +563,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions LENIENT_EXPAND_OPEN_CLOSED_HIDDEN_NO_SELECTOR = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ALLOW_UNAVAILABLE_TARGETS)
@ -650,7 +576,7 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(false)
.allowSelectors(false)
.ignoreThrottled(false)
)
.build();
@ -668,10 +594,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions STRICT_EXPAND_OPEN_CLOSED_HIDDEN = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -682,10 +607,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions STRICT_EXPAND_OPEN_CLOSED_HIDDEN_NO_SELECTORS = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -696,7 +620,7 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(false)
.allowSelectors(false)
.ignoreThrottled(false)
)
.build();
@ -714,10 +638,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.ALL_APPLICABLE)
.build();
public static final IndicesOptions STRICT_EXPAND_OPEN_CLOSED_HIDDEN_FAILURE_STORE = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -728,10 +651,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.ALL_APPLICABLE)
.build();
public static final IndicesOptions STRICT_EXPAND_OPEN_CLOSED_FAILURE_STORE = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -747,10 +669,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(true)
.allowClosedIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.ALL_APPLICABLE)
.build();
public static final IndicesOptions STRICT_EXPAND_OPEN_FORBID_CLOSED = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -766,10 +687,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowClosedIndices(false)
.allowAliasToMultipleIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions STRICT_EXPAND_OPEN_HIDDEN_FORBID_CLOSED = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -785,10 +705,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowClosedIndices(false)
.allowAliasToMultipleIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions STRICT_EXPAND_OPEN_FORBID_CLOSED_IGNORE_THROTTLED = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -804,10 +723,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.ignoreThrottled(true)
.allowClosedIndices(false)
.allowFailureIndices(true)
.allowSelectors(true)
.allowAliasToMultipleIndices(true)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions STRICT_SINGLE_INDEX_NO_EXPAND_FORBID_CLOSED = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -823,10 +741,27 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(false)
.allowClosedIndices(false)
.allowFailureIndices(true)
.allowSelectors(false)
.ignoreThrottled(false)
)
.build();
public static final IndicesOptions STRICT_SINGLE_INDEX_NO_EXPAND_FORBID_CLOSED_ALLOW_SELECTORS = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
.wildcardOptions(
WildcardOptions.builder()
.matchOpen(false)
.matchClosed(false)
.includeHidden(false)
.allowEmptyExpressions(true)
.resolveAliases(true)
)
.gatekeeperOptions(
GatekeeperOptions.builder()
.allowAliasToMultipleIndices(false)
.allowClosedIndices(false)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
public static final IndicesOptions STRICT_NO_EXPAND_FORBID_CLOSED = IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS)
@ -842,10 +777,9 @@ public record IndicesOptions(
GatekeeperOptions.builder()
.allowClosedIndices(false)
.allowAliasToMultipleIndices(true)
.allowFailureIndices(true)
.allowSelectors(true)
.ignoreThrottled(false)
)
.selectorOptions(SelectorOptions.DATA)
.build();
/**
@ -903,10 +837,10 @@ public record IndicesOptions(
}
/**
* @return Whether execution on failure indices is allowed.
* @return Whether selectors (::) are allowed in the index expression.
*/
public boolean allowFailureIndices() {
return gatekeeperOptions.allowFailureIndices();
public boolean allowSelectors() {
return DataStream.isFailureStoreFeatureFlagEnabled() && gatekeeperOptions.allowSelectors();
}
/**
@ -930,20 +864,6 @@ public record IndicesOptions(
return gatekeeperOptions().ignoreThrottled();
}
/**
* @return whether regular indices (stand-alone or backing indices) will be included in the response
*/
public boolean includeRegularIndices() {
return selectorOptions().defaultSelector().shouldIncludeData();
}
/**
* @return whether failure indices (only supported by certain data streams) will be included in the response
*/
public boolean includeFailureIndices() {
return selectorOptions().defaultSelector().shouldIncludeFailures();
}
public void writeIndicesOptions(StreamOutput out) throws IOException {
EnumSet<Option> backwardsCompatibleOptions = EnumSet.noneOf(Option.class);
if (allowNoIndices()) {
@ -964,9 +884,12 @@ public record IndicesOptions(
if (ignoreUnavailable()) {
backwardsCompatibleOptions.add(Option.ALLOW_UNAVAILABLE_CONCRETE_TARGETS);
}
if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0)) {
if (allowFailureIndices()) {
if (allowSelectors()) {
if (out.getTransportVersion()
.between(TransportVersions.V_8_14_0, TransportVersions.REPLACE_FAILURE_STORE_OPTIONS_WITH_SELECTOR_SYNTAX)) {
backwardsCompatibleOptions.add(Option.ALLOW_FAILURE_INDICES);
} else if (out.getTransportVersion().onOrAfter(TransportVersions.REPLACE_FAILURE_STORE_OPTIONS_WITH_SELECTOR_SYNTAX)) {
backwardsCompatibleOptions.add(Option.ALLOW_SELECTORS);
}
}
out.writeEnumSet(backwardsCompatibleOptions);
@ -983,11 +906,17 @@ public record IndicesOptions(
}
out.writeEnumSet(states);
if (out.getTransportVersion().between(TransportVersions.V_8_14_0, TransportVersions.V_8_16_0)) {
out.writeBoolean(includeRegularIndices());
out.writeBoolean(includeFailureIndices());
out.writeBoolean(true);
out.writeBoolean(false);
}
if (out.getTransportVersion()
.between(TransportVersions.V_8_16_0, TransportVersions.REPLACE_FAILURE_STORE_OPTIONS_WITH_SELECTOR_SYNTAX)) {
if (out.getTransportVersion().before(TransportVersions.INTRODUCE_ALL_APPLICABLE_SELECTOR)) {
out.writeVInt(1); // Enum set sized 1
out.writeVInt(0); // ordinal 0 (::data selector)
} else {
out.writeByte((byte) 0); // ordinal 0 (::data selector)
}
if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_16_0)) {
selectorOptions.writeTo(out);
}
}
@ -998,39 +927,44 @@ public record IndicesOptions(
options.contains(Option.ALLOW_EMPTY_WILDCARD_EXPRESSIONS),
options.contains(Option.EXCLUDE_ALIASES)
);
boolean allowFailureIndices = true;
if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0)) {
allowFailureIndices = options.contains(Option.ALLOW_FAILURE_INDICES);
boolean allowSelectors = true;
if (in.getTransportVersion()
.between(TransportVersions.V_8_14_0, TransportVersions.REPLACE_FAILURE_STORE_OPTIONS_WITH_SELECTOR_SYNTAX)) {
// We've effectively replaced the allow failure indices setting with allow selectors. If it is configured on an older version
// then use its value for allow selectors.
allowSelectors = options.contains(Option.ALLOW_FAILURE_INDICES);
} else if (in.getTransportVersion().onOrAfter(TransportVersions.REPLACE_FAILURE_STORE_OPTIONS_WITH_SELECTOR_SYNTAX)) {
allowSelectors = options.contains(Option.ALLOW_SELECTORS);
}
GatekeeperOptions gatekeeperOptions = GatekeeperOptions.builder()
.allowClosedIndices(options.contains(Option.ERROR_WHEN_CLOSED_INDICES) == false)
.allowAliasToMultipleIndices(options.contains(Option.ERROR_WHEN_ALIASES_TO_MULTIPLE_INDICES) == false)
.allowFailureIndices(allowFailureIndices)
.allowSelectors(allowSelectors)
.ignoreThrottled(options.contains(Option.IGNORE_THROTTLED))
.build();
SelectorOptions selectorOptions = SelectorOptions.DEFAULT;
if (in.getTransportVersion().between(TransportVersions.V_8_14_0, TransportVersions.V_8_16_0)) {
// Reading from an older node, which will be sending two booleans that we must read out and ignore.
var includeData = in.readBoolean();
var includeFailures = in.readBoolean();
if (includeData && includeFailures) {
selectorOptions = SelectorOptions.ALL_APPLICABLE;
} else if (includeData) {
selectorOptions = SelectorOptions.DATA;
in.readBoolean();
in.readBoolean();
}
if (in.getTransportVersion()
.between(TransportVersions.V_8_16_0, TransportVersions.REPLACE_FAILURE_STORE_OPTIONS_WITH_SELECTOR_SYNTAX)) {
// Reading from an older node, which will be sending either an enum set or a single byte that needs to be read out and ignored.
if (in.getTransportVersion().before(TransportVersions.INTRODUCE_ALL_APPLICABLE_SELECTOR)) {
int size = in.readVInt();
for (int i = 0; i < size; i++) {
in.readVInt();
}
} else {
selectorOptions = SelectorOptions.FAILURES;
in.readByte();
}
}
if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_16_0)) {
selectorOptions = SelectorOptions.read(in);
}
return new IndicesOptions(
options.contains(Option.ALLOW_UNAVAILABLE_CONCRETE_TARGETS)
? ConcreteTargetOptions.ALLOW_UNAVAILABLE_TARGETS
: ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS,
wildcardOptions,
gatekeeperOptions,
selectorOptions
gatekeeperOptions
);
}
@ -1038,7 +972,6 @@ public record IndicesOptions(
private ConcreteTargetOptions concreteTargetOptions;
private WildcardOptions wildcardOptions;
private GatekeeperOptions gatekeeperOptions;
private SelectorOptions selectorOptions;
Builder() {
this(DEFAULT);
@ -1048,7 +981,6 @@ public record IndicesOptions(
concreteTargetOptions = indicesOptions.concreteTargetOptions;
wildcardOptions = indicesOptions.wildcardOptions;
gatekeeperOptions = indicesOptions.gatekeeperOptions;
selectorOptions = indicesOptions.selectorOptions;
}
public Builder concreteTargetOptions(ConcreteTargetOptions concreteTargetOptions) {
@ -1076,13 +1008,8 @@ public record IndicesOptions(
return this;
}
public Builder selectorOptions(SelectorOptions selectorOptions) {
this.selectorOptions = selectorOptions;
return this;
}
public IndicesOptions build() {
return new IndicesOptions(concreteTargetOptions, wildcardOptions, gatekeeperOptions, selectorOptions);
return new IndicesOptions(concreteTargetOptions, wildcardOptions, gatekeeperOptions);
}
}
@ -1181,12 +1108,10 @@ public record IndicesOptions(
.allowClosedIndices(forbidClosedIndices == false)
.ignoreThrottled(ignoreThrottled)
.build();
final SelectorOptions selectorOptions = SelectorOptions.DEFAULT;
return new IndicesOptions(
ignoreUnavailable ? ConcreteTargetOptions.ALLOW_UNAVAILABLE_TARGETS : ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS,
wildcards,
gatekeeperOptions,
selectorOptions
gatekeeperOptions
);
}
@ -1200,7 +1125,6 @@ public record IndicesOptions(
request.param(ConcreteTargetOptions.IGNORE_UNAVAILABLE),
request.param(WildcardOptions.ALLOW_NO_INDICES),
request.param(GatekeeperOptions.IGNORE_THROTTLED),
DataStream.isFailureStoreFeatureFlagEnabled() ? request.param(FAILURE_STORE_QUERY_PARAM) : INCLUDE_ONLY_REGULAR_INDICES,
defaultSettings
);
}
@ -1216,7 +1140,6 @@ public record IndicesOptions(
map.containsKey(GatekeeperOptions.IGNORE_THROTTLED)
? map.get(GatekeeperOptions.IGNORE_THROTTLED)
: map.get("ignoreThrottled"),
map.containsKey(FAILURE_STORE_QUERY_PARAM) ? map.get(FAILURE_STORE_QUERY_PARAM) : map.get("failureStore"),
defaultSettings
);
}
@ -1243,9 +1166,7 @@ public record IndicesOptions(
|| GatekeeperOptions.IGNORE_THROTTLED.equals(name)
|| "ignoreThrottled".equals(name)
|| WildcardOptions.ALLOW_NO_INDICES.equals(name)
|| "allowNoIndices".equals(name)
|| (DataStream.isFailureStoreFeatureFlagEnabled() && FAILURE_STORE_QUERY_PARAM.equals(name))
|| (DataStream.isFailureStoreFeatureFlagEnabled() && "failureStore".equals(name));
|| "allowNoIndices".equals(name);
}
public static IndicesOptions fromParameters(
@ -1276,52 +1197,20 @@ public record IndicesOptions(
WildcardOptions wildcards = WildcardOptions.parseParameters(wildcardsString, allowNoIndicesString, defaultSettings.wildcardOptions);
GatekeeperOptions gatekeeperOptions = GatekeeperOptions.parseParameter(ignoreThrottled, defaultSettings.gatekeeperOptions);
SelectorOptions selectorOptions = DataStream.isFailureStoreFeatureFlagEnabled()
? parseFailureStoreParameters(failureStoreString, defaultSettings.selectorOptions)
: SelectorOptions.DEFAULT;
// note that allowAliasesToMultipleIndices is not exposed, always true (only for internal use)
return IndicesOptions.builder()
.concreteTargetOptions(ConcreteTargetOptions.fromParameter(ignoreUnavailableString, defaultSettings.concreteTargetOptions))
.wildcardOptions(wildcards)
.gatekeeperOptions(gatekeeperOptions)
.selectorOptions(selectorOptions)
.build();
}
/**
* @deprecated This method parses the query parameter failure_store. This is a deprecated param, and it will be replaced
* the selector suffix, for example `my-data-stream::data` or `my-data-stream::failures`
*/
@Deprecated
private static SelectorOptions parseFailureStoreParameters(Object failureStoreValue, SelectorOptions defaultOptions) {
if (failureStoreValue == null) {
return defaultOptions;
}
return switch (failureStoreValue.toString()) {
case INCLUDE_ALL -> SelectorOptions.ALL_APPLICABLE;
case INCLUDE_ONLY_REGULAR_INDICES -> SelectorOptions.DATA;
case INCLUDE_ONLY_FAILURE_INDICES -> SelectorOptions.FAILURES;
default -> throw new IllegalArgumentException("No valid " + FAILURE_STORE_QUERY_PARAM + " value [" + failureStoreValue + "]");
};
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException {
concreteTargetOptions.toXContent(builder, params);
wildcardOptions.toXContent(builder, params);
gatekeeperOptions.toXContent(builder, params);
if (DataStream.isFailureStoreFeatureFlagEnabled()) {
String displayValue;
if (SelectorOptions.ALL_APPLICABLE.equals(selectorOptions())) {
displayValue = INCLUDE_ALL;
} else if (SelectorOptions.DATA.equals(selectorOptions())) {
displayValue = INCLUDE_ONLY_REGULAR_INDICES;
} else {
displayValue = INCLUDE_ONLY_FAILURE_INDICES;
}
builder.field(FAILURE_STORE_QUERY_PARAM, displayValue);
}
return builder;
}
@ -1329,7 +1218,6 @@ public record IndicesOptions(
private static final ParseField IGNORE_UNAVAILABLE_FIELD = new ParseField(ConcreteTargetOptions.IGNORE_UNAVAILABLE);
private static final ParseField IGNORE_THROTTLED_FIELD = new ParseField(GatekeeperOptions.IGNORE_THROTTLED).withAllDeprecated();
private static final ParseField ALLOW_NO_INDICES_FIELD = new ParseField(WildcardOptions.ALLOW_NO_INDICES);
private static final ParseField FAILURE_STORE_FIELD = new ParseField(FAILURE_STORE_QUERY_PARAM);
public static IndicesOptions fromXContent(XContentParser parser) throws IOException {
return fromXContent(parser, null);
@ -1340,7 +1228,6 @@ public record IndicesOptions(
WildcardOptions.Builder wildcards = defaults == null ? null : WildcardOptions.builder(defaults.wildcardOptions());
GatekeeperOptions.Builder generalOptions = GatekeeperOptions.builder()
.ignoreThrottled(defaults != null && defaults.gatekeeperOptions().ignoreThrottled());
SelectorOptions selectorOptions = defaults == null ? SelectorOptions.DEFAULT : defaults.selectorOptions();
Boolean allowNoIndices = defaults == null ? null : defaults.allowNoIndices();
Boolean ignoreUnavailable = defaults == null ? null : defaults.ignoreUnavailable();
Token token = parser.currentToken() == Token.START_OBJECT ? parser.currentToken() : parser.nextToken();
@ -1390,9 +1277,6 @@ public record IndicesOptions(
allowNoIndices = parser.booleanValue();
} else if (IGNORE_THROTTLED_FIELD.match(currentFieldName, parser.getDeprecationHandler())) {
generalOptions.ignoreThrottled(parser.booleanValue());
} else if (DataStream.isFailureStoreFeatureFlagEnabled()
&& FAILURE_STORE_FIELD.match(currentFieldName, parser.getDeprecationHandler())) {
selectorOptions = parseFailureStoreParameters(parser.text(), selectorOptions);
} else {
throw new ElasticsearchParseException(
"could not read indices options. Unexpected index option [" + currentFieldName + "]"
@ -1423,7 +1307,6 @@ public record IndicesOptions(
.concreteTargetOptions(new ConcreteTargetOptions(ignoreUnavailable))
.wildcardOptions(wildcards)
.gatekeeperOptions(generalOptions)
.selectorOptions(selectorOptions)
.build();
}
@ -1435,14 +1318,6 @@ public record IndicesOptions(
return STRICT_EXPAND_OPEN;
}
/**
* @return indices options that requires every specified index to exist, expands wildcards only to open indices and
* allows that no indices are resolved from wildcard expressions (not returning an error).
*/
public static IndicesOptions strictExpandOpenIncludeFailureStore() {
return STRICT_EXPAND_OPEN_FAILURE_STORE;
}
/**
* @return indices options that requires every specified index to exist, expands wildcards only to open indices,
* allows that no indices are resolved from wildcard expressions (not returning an error) and forbids the
@ -1477,6 +1352,15 @@ public record IndicesOptions(
return STRICT_EXPAND_OPEN_CLOSED_HIDDEN;
}
/**
* @return indices option that requires every specified index to exist, expands wildcards to both open and closed indices, includes
* hidden indices, allows that no indices are resolved from wildcard expressions (not returning an error), and disallows selectors
* in the expression (no :: separators).
*/
public static IndicesOptions strictExpandHiddenNoSelectors() {
return STRICT_EXPAND_OPEN_CLOSED_HIDDEN_NO_SELECTORS;
}
/**
* @return indices option that expands wildcards to both open and closed indices, includes failure store
* (with data stream) and allows that indices can be missing and no indices are resolved from wildcard expressions
@ -1510,6 +1394,14 @@ public record IndicesOptions(
return STRICT_SINGLE_INDEX_NO_EXPAND_FORBID_CLOSED;
}
/**
* @return indices option that requires each specified index or alias to exist, doesn't expand wildcards and
* throws error if any of the aliases resolves to multiple indices
*/
public static IndicesOptions strictSingleIndexNoExpandForbidClosedAllowSelectors() {
return STRICT_SINGLE_INDEX_NO_EXPAND_FORBID_CLOSED_ALLOW_SELECTORS;
}
/**
* @return indices options that ignores unavailable indices, expands wildcards only to open indices and
* allows that no indices are resolved from wildcard expressions (not returning an error).
@ -1518,6 +1410,15 @@ public record IndicesOptions(
return LENIENT_EXPAND_OPEN;
}
/**
* @return indices options that ignores unavailable indices, expands wildcards only to open indices,
* allows that no indices are resolved from wildcard expressions (not returning an error), and disallows
* selectors in the expression (no :: separators).
*/
public static IndicesOptions lenientExpandOpenNoSelectors() {
return LENIENT_EXPAND_OPEN_NO_SELECTORS;
}
/**
* @return indices options that ignores unavailable indices, expands wildcards to open and hidden indices, and
* allows that no indices are resolved from wildcard expressions (not returning an error).
@ -1563,14 +1464,7 @@ public record IndicesOptions(
+ ignoreAliases()
+ ", ignore_throttled="
+ ignoreThrottled()
+ (DataStream.isFailureStoreFeatureFlagEnabled()
? ", include_regular_indices="
+ includeRegularIndices()
+ ", include_failure_indices="
+ includeFailureIndices()
+ ", allow_failure_indices="
+ allowFailureIndices()
: "")
+ (DataStream.isFailureStoreFeatureFlagEnabled() ? ", allow_selectors=" + allowSelectors() : "")
+ ']';
}
}

View file

@ -9,8 +9,11 @@
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.Tuple;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.indices.SystemIndices.SystemIndexAccessLevel;
@ -49,6 +52,18 @@ public class IndexAbstractionResolver {
indexAbstraction = index;
}
// Always check to see if there's a selector on the index expression
Tuple<String, String> expressionAndSelector = IndexNameExpressionResolver.splitSelectorExpression(indexAbstraction);
String selectorString = expressionAndSelector.v2();
if (indicesOptions.allowSelectors() == false && selectorString != null) {
throw new IllegalArgumentException(
"Index component selectors are not supported in this context but found selector in expression ["
+ indexAbstraction
+ "]"
);
}
indexAbstraction = expressionAndSelector.v1();
// we always need to check for date math expressions
indexAbstraction = IndexNameExpressionResolver.resolveDateMathExpression(indexAbstraction);
@ -59,13 +74,15 @@ public class IndexAbstractionResolver {
if (Regex.simpleMatch(indexAbstraction, authorizedIndex)
&& isIndexVisible(
indexAbstraction,
selectorString,
authorizedIndex,
indicesOptions,
metadata,
indexNameExpressionResolver,
includeDataStreams
)) {
resolvedIndices.add(authorizedIndex);
// Resolve any ::* suffixes on the expression. We need to resolve them all to their final valid selectors
resolveSelectorsAndCombine(authorizedIndex, selectorString, indicesOptions, resolvedIndices, metadata);
}
}
if (resolvedIndices.isEmpty()) {
@ -81,21 +98,63 @@ public class IndexAbstractionResolver {
}
}
} else {
// Resolve any ::* suffixes on the expression. We need to resolve them all to their final valid selectors
Set<String> resolvedIndices = new HashSet<>();
resolveSelectorsAndCombine(indexAbstraction, selectorString, indicesOptions, resolvedIndices, metadata);
if (minus) {
finalIndices.remove(indexAbstraction);
finalIndices.removeAll(resolvedIndices);
} else if (indicesOptions.ignoreUnavailable() == false || isAuthorized.test(indexAbstraction)) {
// Unauthorized names are considered unavailable, so if `ignoreUnavailable` is `true` they should be silently
// discarded from the `finalIndices` list. Other "ways of unavailable" must be handled by the action
// handler, see: https://github.com/elastic/elasticsearch/issues/90215
finalIndices.add(indexAbstraction);
finalIndices.addAll(resolvedIndices);
}
}
}
return finalIndices;
}
private static void resolveSelectorsAndCombine(
String indexAbstraction,
String selectorString,
IndicesOptions indicesOptions,
Set<String> collect,
Metadata metadata
) {
if (indicesOptions.allowSelectors()) {
IndexAbstraction abstraction = metadata.getIndicesLookup().get(indexAbstraction);
// We can't determine which selectors are valid for a nonexistent abstraction, so simply propagate them as if they supported
// all of them so we don't drop anything.
boolean acceptsAllSelectors = abstraction == null || abstraction.isDataStreamRelated();
// Supply default if needed
if (selectorString == null) {
selectorString = IndexComponentSelector.DATA.getKey();
}
if (Regex.isMatchAllPattern(selectorString)) {
// Always accept data
collect.add(IndexNameExpressionResolver.combineSelectorExpression(indexAbstraction, IndexComponentSelector.DATA.getKey()));
// Only put failures on the expression if the abstraction supports it.
if (acceptsAllSelectors) {
collect.add(
IndexNameExpressionResolver.combineSelectorExpression(indexAbstraction, IndexComponentSelector.FAILURES.getKey())
);
}
} else {
// A non-wildcard selector is always passed along as-is, it's validity for this kind of abstraction is tested later
collect.add(IndexNameExpressionResolver.combineSelectorExpression(indexAbstraction, selectorString));
}
} else {
assert selectorString == null
: "A selector string [" + selectorString + "] is present but selectors are disabled in this context";
collect.add(indexAbstraction);
}
}
public static boolean isIndexVisible(
String expression,
@Nullable String selectorString,
String index,
IndicesOptions indicesOptions,
Metadata metadata,
@ -111,7 +170,15 @@ public class IndexAbstractionResolver {
if (indexAbstraction.getType() == IndexAbstraction.Type.ALIAS) {
// it's an alias, ignore expandWildcardsOpen and expandWildcardsClosed.
// complicated to support those options with aliases pointing to multiple indices...
return isVisible && indicesOptions.ignoreAliases() == false;
isVisible = isVisible && indicesOptions.ignoreAliases() == false;
if (isVisible && selectorString != null) {
// Check if a selector was present, and if it is, check if this alias is applicable to it
IndexComponentSelector selector = IndexComponentSelector.getByKey(selectorString);
if (IndexComponentSelector.FAILURES.equals(selector)) {
isVisible = indexAbstraction.isDataStreamRelated();
}
}
return isVisible;
}
if (indexAbstraction.getType() == IndexAbstraction.Type.DATA_STREAM) {
if (includeDataStreams == false) {
@ -142,6 +209,13 @@ public class IndexAbstractionResolver {
return isSystemIndexVisible(resolver, indexAbstraction);
}
}
if (selectorString != null && Regex.isMatchAllPattern(selectorString) == false) {
// Check if a selector was present, and if it is, check if this index is applicable to it
IndexComponentSelector selector = IndexComponentSelector.getByKey(selectorString);
if (IndexComponentSelector.FAILURES.equals(selector)) {
return false;
}
}
IndexMetadata indexMetadata = metadata.index(indexAbstraction.getIndices().get(0));
if (indexMetadata.getState() == IndexMetadata.State.CLOSE && indicesOptions.expandWildcardsClosed()) {

View file

@ -11,7 +11,6 @@ package org.elasticsearch.rest.action.admin.indices;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.rest.BaseRestHandler;
@ -45,7 +44,7 @@ public class RestRolloverIndexAction extends BaseRestHandler {
@Override
public Set<String> supportedCapabilities() {
if (DataStream.isFailureStoreFeatureFlagEnabled()) {
return Set.of("lazy-rollover-failure-store");
return Set.of("lazy-rollover-failure-store", "index-expression-selectors");
} else {
return Set.of();
}
@ -59,16 +58,6 @@ public class RestRolloverIndexAction extends BaseRestHandler {
rolloverIndexRequest.lazy(request.paramAsBoolean("lazy", false));
rolloverIndexRequest.ackTimeout(getAckTimeout(request));
rolloverIndexRequest.masterNodeTimeout(getMasterNodeTimeout(request));
if (DataStream.isFailureStoreFeatureFlagEnabled()) {
boolean failureStore = request.paramAsBoolean("target_failure_store", false);
if (failureStore) {
rolloverIndexRequest.setIndicesOptions(
IndicesOptions.builder(rolloverIndexRequest.indicesOptions())
.selectorOptions(IndicesOptions.SelectorOptions.FAILURES)
.build()
);
}
}
rolloverIndexRequest.getCreateIndexRequest()
.waitForActiveShards(ActiveShardCount.parseString(request.param("wait_for_active_shards")));
return channel -> new RestCancellableNodeClient(client, request.getHttpChannel()).admin()

View file

@ -362,6 +362,7 @@ public final class RestoreService implements ClusterStateApplier {
metadataBuilder = Metadata.builder();
}
// TODO: https://github.com/elastic/elasticsearch/issues/119545 - This does not yet support selectors
final String[] indicesInRequest = request.indices();
List<String> requestIndices = new ArrayList<>(indicesInRequest.length);
if (indicesInRequest.length == 0) {

View file

@ -4137,6 +4137,9 @@ public final class SnapshotsService extends AbstractLifecycleComponent implement
request.partial(),
indexIds,
CollectionUtils.concatLists(
// It's ok to just get the data stream names here because we have already resolved every concrete index that will be
// in the snapshot, and thus already resolved any selectors that might be present. We now only care about which data
// streams we're packing up in the resulting snapshot, not what their contents are.
indexNameExpressionResolver.dataStreamNames(currentState, request.indicesOptions(), request.indices()),
systemDataStreamNames
),

View file

@ -10,11 +10,14 @@
package org.elasticsearch.transport;
import org.elasticsearch.cluster.metadata.ClusterNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SelectorResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.core.Strings;
import org.elasticsearch.core.Tuple;
import org.elasticsearch.node.Node;
import java.util.ArrayList;
@ -64,8 +67,11 @@ public abstract class RemoteClusterAware {
// Thus, whatever it is, this is definitely not a remote index.
return false;
}
int idx = indexExpression.indexOf(RemoteClusterService.REMOTE_CLUSTER_INDEX_SEPARATOR);
// Check to make sure the remote cluster separator ':' isn't actually a selector separator '::'
boolean isSelector = indexExpression.startsWith(SelectorResolver.SELECTOR_SEPARATOR, idx);
// Note remote index name also can not start with ':'
return indexExpression.indexOf(RemoteClusterService.REMOTE_CLUSTER_INDEX_SEPARATOR) > 0;
return idx > 0 && isSelector == false;
}
/**
@ -99,7 +105,8 @@ public abstract class RemoteClusterAware {
if (i == 0) {
throw new IllegalArgumentException("index name [" + indexExpression + "] is invalid because the remote part is empty");
}
if (i < 0) {
if (i < 0 || indexExpression.startsWith(SelectorResolver.SELECTOR_SEPARATOR, i)) {
// Either no colon present, or the colon was a part of a selector separator (::)
return new String[] { null, indexExpression };
} else {
return new String[] { indexExpression.substring(0, i), indexExpression.substring(i + 1) };
@ -144,6 +151,9 @@ public abstract class RemoteClusterAware {
isNegative ? remoteClusterName.substring(1) : remoteClusterName
);
if (isNegative) {
Tuple<String, String> indexAndSelector = IndexNameExpressionResolver.splitSelectorExpression(indexName);
indexName = indexAndSelector.v1();
String selectorString = indexAndSelector.v2();
if (indexName.equals("*") == false) {
throw new IllegalArgumentException(
Strings.format(
@ -152,6 +162,14 @@ public abstract class RemoteClusterAware {
)
);
}
if (selectorString != null && selectorString.equals("*") == false) {
throw new IllegalArgumentException(
Strings.format(
"To exclude a cluster you must specify the '::*' selector or leave it off, but found: [%s]",
selectorString
)
);
}
clustersToRemove.addAll(clusters);
} else {
for (String clusterName : clusters) {

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action;
import org.elasticsearch.TransportVersions;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
@ -42,9 +43,14 @@ public class OriginalIndicesTests extends ESTestCase {
OriginalIndices originalIndices2 = OriginalIndices.readOriginalIndices(in);
assertThat(originalIndices2.indices(), equalTo(originalIndices.indices()));
// indices options are not equivalent when sent to an older version and re-read due
// to the addition of selector settings. Allow selectors is always true when read
// from a version prior to its addition, since true is the default value.
if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0) || originalIndices.indicesOptions().allowSelectors()) {
assertThat(originalIndices2.indicesOptions(), equalTo(originalIndices.indicesOptions()));
}
}
}
public static OriginalIndices randomOriginalIndices() {
int numIndices = randomInt(10);

View file

@ -82,6 +82,5 @@ public class GetIndexRequestTests extends ESTestCase {
);
assertThat(getIndexRequest.indicesOptions().wildcardOptions(), equalTo(IndicesOptions.strictExpandOpen().wildcardOptions()));
assertThat(getIndexRequest.indicesOptions().gatekeeperOptions(), equalTo(IndicesOptions.strictExpandOpen().gatekeeperOptions()));
assertThat(getIndexRequest.indicesOptions().selectorOptions(), equalTo(IndicesOptions.SelectorOptions.ALL_APPLICABLE));
}
}

View file

@ -14,6 +14,7 @@ import org.elasticsearch.action.admin.indices.resolve.ResolveIndexAction.Resolve
import org.elasticsearch.action.admin.indices.resolve.ResolveIndexAction.ResolvedDataStream;
import org.elasticsearch.action.admin.indices.resolve.ResolveIndexAction.ResolvedIndex;
import org.elasticsearch.action.admin.indices.resolve.ResolveIndexAction.TransportAction;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
@ -240,7 +241,10 @@ public class ResolveIndexTests extends ESTestCase {
assertThat(
resolvedIndices,
contains(
oneOf(new ResolvedExpression("logs-pgsql-prod-" + todaySuffix), new ResolvedExpression("logs-pgsql-prod-" + tomorrowSuffix))
oneOf(
new ResolvedExpression("logs-pgsql-prod-" + todaySuffix, IndexComponentSelector.DATA),
new ResolvedExpression("logs-pgsql-prod-" + tomorrowSuffix, IndexComponentSelector.DATA)
)
)
);
}

View file

@ -13,7 +13,6 @@ import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.AliasAction;
@ -747,15 +746,15 @@ public class MetadataRolloverServiceTests extends ESTestCase {
final String defaultRolloverIndexName;
final boolean useDataStream = randomBoolean();
final Metadata.Builder builder = Metadata.builder();
var defaultSelectorOptions = IndicesOptions.SelectorOptions.DEFAULT;
boolean isFailureStoreRollover = false;
if (useDataStream) {
DataStream dataStream = DataStreamTestHelper.randomInstance()
// ensure no replicate data stream
.promoteDataStream();
rolloverTarget = dataStream.getName();
if (dataStream.isFailureStoreExplicitlyEnabled() && randomBoolean()) {
defaultSelectorOptions = IndicesOptions.SelectorOptions.FAILURES;
sourceIndexName = dataStream.getFailureStoreWriteIndex().getName();
isFailureStoreRollover = true;
defaultRolloverIndexName = DataStream.getDefaultFailureStoreName(
dataStream.getName(),
dataStream.getGeneration() + 1,
@ -815,7 +814,7 @@ public class MetadataRolloverServiceTests extends ESTestCase {
true,
null,
null,
IndicesOptions.SelectorOptions.FAILURES.equals(defaultSelectorOptions)
isFailureStoreRollover
);
newIndexName = newIndexName == null ? defaultRolloverIndexName : newIndexName;

View file

@ -11,7 +11,6 @@ package org.elasticsearch.action.admin.indices.rollover;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
@ -174,9 +173,6 @@ public class RolloverRequestTests extends ESTestCase {
.build()
);
originalRequest.lazy(randomBoolean());
originalRequest.setIndicesOptions(
IndicesOptions.builder(originalRequest.indicesOptions()).selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE).build()
);
try (BytesStreamOutput out = new BytesStreamOutput()) {
originalRequest.writeTo(out);
@ -186,7 +182,6 @@ public class RolloverRequestTests extends ESTestCase {
assertThat(cloneRequest.getNewIndexName(), equalTo(originalRequest.getNewIndexName()));
assertThat(cloneRequest.getRolloverTarget(), equalTo(originalRequest.getRolloverTarget()));
assertThat(cloneRequest.isLazy(), equalTo(originalRequest.isLazy()));
assertThat(cloneRequest.indicesOptions().selectorOptions(), equalTo(originalRequest.indicesOptions().selectorOptions()));
for (Map.Entry<String, Condition<?>> entry : cloneRequest.getConditions().getConditions().entrySet()) {
Condition<?> condition = originalRequest.getConditions().getConditions().get(entry.getKey());
// here we compare the string representation as there is some information loss when serializing
@ -253,12 +248,7 @@ public class RolloverRequestTests extends ESTestCase {
}
{
RolloverRequest rolloverRequest = new RolloverRequest("alias-index", "new-index-name");
rolloverRequest.setIndicesOptions(
IndicesOptions.builder(rolloverRequest.indicesOptions())
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.build()
);
RolloverRequest rolloverRequest = new RolloverRequest("alias-index::*", "new-index-name");
ActionRequestValidationException validationException = rolloverRequest.validate();
assertNotNull(validationException);
assertEquals(1, validationException.validationErrors().size());

View file

@ -22,6 +22,7 @@ import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.ActionTestUtils;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.client.internal.node.NodeClient;
@ -34,6 +35,7 @@ import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexAbstraction.ConcreteIndex;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.Template;
import org.elasticsearch.cluster.node.DiscoveryNode;
@ -137,9 +139,11 @@ public class TransportBulkActionTests extends ESTestCase {
@Override
void rollOver(RolloverRequest rolloverRequest, ActionListener<RolloverResponse> listener) {
if (failDataStreamRolloverException != null && rolloverRequest.targetsFailureStore() == false) {
String selectorString = IndexNameExpressionResolver.splitSelectorExpression(rolloverRequest.getRolloverTarget()).v2();
boolean isFailureStoreRollover = IndexComponentSelector.FAILURES.getKey().equals(selectorString);
if (failDataStreamRolloverException != null && isFailureStoreRollover == false) {
listener.onFailure(failDataStreamRolloverException);
} else if (failFailureStoreRolloverException != null && rolloverRequest.targetsFailureStore()) {
} else if (failFailureStoreRolloverException != null && isFailureStoreRollover) {
listener.onFailure(failFailureStoreRolloverException);
} else {
listener.onResponse(

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action.datastreams;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
@ -17,6 +18,7 @@ import org.elasticsearch.cluster.metadata.DataStreamMetadata;
import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.settings.Settings;
@ -46,6 +48,8 @@ public class DataStreamsActionUtilTests extends ESTestCase {
var dataStreamIndex2 = new Index(".ds-bar2", IndexMetadata.INDEX_UUID_NA_VALUE);
var dataStreamIndex3 = new Index(".ds-foo2", IndexMetadata.INDEX_UUID_NA_VALUE);
var dataStreamIndex4 = new Index(".ds-baz1", IndexMetadata.INDEX_UUID_NA_VALUE);
var dataStreamFailureIndex1 = new Index(".fs-foo1", IndexMetadata.INDEX_UUID_NA_VALUE);
var dataStreamFailureIndex2 = new Index(".fs-bar2", IndexMetadata.INDEX_UUID_NA_VALUE);
ClusterState clusterState = ClusterState.builder(new ClusterName("test-cluster"))
.metadata(
@ -54,8 +58,14 @@ public class DataStreamsActionUtilTests extends ESTestCase {
DataStreamMetadata.TYPE,
new DataStreamMetadata(
ImmutableOpenMap.<String, DataStream>builder()
.fPut("fooDs", DataStreamTestHelper.newInstance("fooDs", List.of(dataStreamIndex1)))
.fPut("barDs", DataStreamTestHelper.newInstance("barDs", List.of(dataStreamIndex2)))
.fPut(
"fooDs",
DataStreamTestHelper.newInstance("fooDs", List.of(dataStreamIndex1), List.of(dataStreamFailureIndex1))
)
.fPut(
"barDs",
DataStreamTestHelper.newInstance("barDs", List.of(dataStreamIndex2), List.of(dataStreamFailureIndex2))
)
.fPut("foo2Ds", DataStreamTestHelper.newInstance("foo2Ds", List.of(dataStreamIndex3)))
.fPut("bazDs", DataStreamTestHelper.newInstance("bazDs", List.of(dataStreamIndex4)))
.build(),
@ -69,7 +79,9 @@ public class DataStreamsActionUtilTests extends ESTestCase {
dataStreamIndex1,
dataStreamIndex2,
dataStreamIndex3,
dataStreamIndex4
dataStreamIndex4,
dataStreamFailureIndex1,
dataStreamFailureIndex2
)
)
.build()
@ -78,16 +90,39 @@ public class DataStreamsActionUtilTests extends ESTestCase {
var query = new String[] { "foo*", "baz*" };
var indexNameExpressionResolver = mock(IndexNameExpressionResolver.class);
when(indexNameExpressionResolver.dataStreamNames(any(), any(), eq(query))).thenReturn(List.of("fooDs", "foo2Ds", "bazDs"));
when(indexNameExpressionResolver.dataStreams(any(), any(), eq(query))).thenReturn(
List.of(new ResolvedExpression("fooDs"), new ResolvedExpression("foo2Ds"), new ResolvedExpression("bazDs"))
);
var resolved = DataStreamsActionUtil.resolveConcreteIndexNames(
indexNameExpressionResolver,
clusterState,
query,
IndicesOptions.builder().wildcardOptions(IndicesOptions.WildcardOptions.builder().includeHidden(true)).build()
).toList();
);
assertThat(resolved, containsInAnyOrder(".ds-foo1", ".ds-foo2", ".ds-baz1"));
when(indexNameExpressionResolver.dataStreams(any(), any(), eq(query))).thenReturn(
List.of(
new ResolvedExpression("fooDs", IndexComponentSelector.DATA),
new ResolvedExpression("fooDs", IndexComponentSelector.FAILURES),
new ResolvedExpression("foo2Ds", IndexComponentSelector.DATA),
new ResolvedExpression("foo2Ds", IndexComponentSelector.FAILURES),
new ResolvedExpression("bazDs", IndexComponentSelector.DATA),
new ResolvedExpression("bazDs", IndexComponentSelector.FAILURES)
)
);
resolved = DataStreamsActionUtil.resolveConcreteIndexNames(
indexNameExpressionResolver,
clusterState,
query,
IndicesOptions.builder().wildcardOptions(IndicesOptions.WildcardOptions.builder().includeHidden(true)).build()
);
assertThat(resolved, containsInAnyOrder(".ds-foo1", ".fs-foo1", ".ds-foo2", ".ds-baz1"));
}
private Map<String, IndexMetadata> createLocalOnlyIndicesMetadata(Index... indices) {

View file

@ -56,7 +56,6 @@ public class IndicesOptionsTests extends ESTestCase {
.allowAliasToMultipleIndices(randomBoolean())
.allowClosedIndices(randomBoolean())
)
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.build();
BytesStreamOutput output = new BytesStreamOutput();
@ -342,9 +341,8 @@ public class IndicesOptionsTests extends ESTestCase {
randomBoolean()
);
GatekeeperOptions gatekeeperOptions = new GatekeeperOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean());
IndicesOptions.SelectorOptions selectorOptions = new IndicesOptions.SelectorOptions(randomFrom(IndexComponentSelector.values()));
IndicesOptions indicesOptions = new IndicesOptions(concreteTargetOptions, wildcardOptions, gatekeeperOptions, selectorOptions);
IndicesOptions indicesOptions = new IndicesOptions(concreteTargetOptions, wildcardOptions, gatekeeperOptions);
XContentType type = randomFrom(XContentType.values());
BytesReference xContentBytes = toXContentBytes(indicesOptions, type);
@ -359,15 +357,6 @@ public class IndicesOptionsTests extends ESTestCase {
assertThat(map.get("ignore_unavailable"), equalTo(concreteTargetOptions.allowUnavailableTargets()));
assertThat(map.get("allow_no_indices"), equalTo(wildcardOptions.allowEmptyExpressions()));
assertThat(map.get("ignore_throttled"), equalTo(gatekeeperOptions.ignoreThrottled()));
String displayValue;
if (IndicesOptions.SelectorOptions.ALL_APPLICABLE.equals(selectorOptions)) {
displayValue = "include";
} else if (IndicesOptions.SelectorOptions.DATA.equals(selectorOptions)) {
displayValue = "exclude";
} else {
displayValue = "only";
}
assertThat(map.get("failure_store"), equalTo(displayValue));
}
public void testFromXContent() throws IOException {

View file

@ -0,0 +1,241 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.core.Tuple;
import org.elasticsearch.indices.EmptySystemIndices;
import org.elasticsearch.indices.InvalidIndexNameException;
import org.elasticsearch.test.ESTestCase;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.either;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
public class IndexAbstractionResolverTests extends ESTestCase {
private IndexNameExpressionResolver indexNameExpressionResolver;
private IndexAbstractionResolver indexAbstractionResolver;
private Metadata metadata;
private String dateTimeIndexToday;
private String dateTimeIndexTomorrow;
// Only used when resolving wildcard expressions
private final Supplier<Set<String>> defaultMask = () -> Set.of("index1", "index2", "data-stream1");
@Override
public void setUp() throws Exception {
super.setUp();
indexNameExpressionResolver = new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY), EmptySystemIndices.INSTANCE);
indexAbstractionResolver = new IndexAbstractionResolver(indexNameExpressionResolver);
// Try to resist failing at midnight on the first/last day of the month. Time generally moves forward, so make a timestamp for
// the next day and if they're different, add both to the cluster state. Check for either in date math tests.
long timeMillis = System.currentTimeMillis();
long timeTomorrow = timeMillis + TimeUnit.DAYS.toMillis(1);
dateTimeIndexToday = IndexNameExpressionResolver.resolveDateMathExpression("<datetime-{now/M}>", timeMillis);
dateTimeIndexTomorrow = IndexNameExpressionResolver.resolveDateMathExpression("<datetime-{now/M}>", timeTomorrow);
metadata = DataStreamTestHelper.getClusterStateWithDataStreams(
List.of(new Tuple<>("data-stream1", 2), new Tuple<>("data-stream2", 2)),
List.of("index1", "index2", "index3", dateTimeIndexToday, dateTimeIndexTomorrow),
randomMillisUpToYear9999(),
Settings.EMPTY,
0,
false,
true
).metadata();
}
public void testResolveIndexAbstractions() {
// == Single Concrete Index ==
// No selectors allowed, none given
assertThat(resolveAbstractionsSelectorNotAllowed(List.of("index1")), contains("index1"));
// No selectors allowed, valid selector given
expectThrows(IllegalArgumentException.class, () -> resolveAbstractionsSelectorNotAllowed(List.of("index1::data")));
// Selectors allowed, valid selector given, data selector stripped off in result since it is the default
assertThat(resolveAbstractionsSelectorAllowed(List.of("index1::data")), contains("index1"));
// Selectors allowed, wildcard selector provided, data selector stripped off in result since it is the default
// ** only returns ::data since expression is an index
assertThat(resolveAbstractionsSelectorAllowed(List.of("index1::*")), contains("index1"));
// Selectors allowed, invalid selector given
expectThrows(InvalidIndexNameException.class, () -> resolveAbstractionsSelectorAllowed(List.of("index1::custom")));
// == Single Date Math Expressions ==
// No selectors allowed, none given
assertThat(
resolveAbstractionsSelectorNotAllowed(List.of("<datetime-{now/M}>")),
contains(either(equalTo(dateTimeIndexToday)).or(equalTo(dateTimeIndexTomorrow)))
);
// No selectors allowed, valid selector given
expectThrows(IllegalArgumentException.class, () -> resolveAbstractionsSelectorNotAllowed(List.of("<datetime-{now/M}>::data")));
// Selectors allowed, none given
assertThat(
resolveAbstractionsSelectorAllowed(List.of("<datetime-{now/M}>")),
contains(either(equalTo(dateTimeIndexToday)).or(equalTo(dateTimeIndexTomorrow)))
);
// Selectors allowed, valid selector provided, data selector stripped off in result since it is the default
assertThat(
resolveAbstractionsSelectorAllowed(List.of("<datetime-{now/M}>::data")),
contains(either(equalTo(dateTimeIndexToday)).or(equalTo(dateTimeIndexTomorrow)))
);
// Selectors allowed, wildcard selector provided, data selector stripped off in result since it is the default
// ** only returns ::data since expression is an index
assertThat(
resolveAbstractionsSelectorAllowed(List.of("<datetime-{now/M}>::data")),
contains(either(equalTo(dateTimeIndexToday)).or(equalTo(dateTimeIndexTomorrow)))
);
// Selectors allowed, invalid selector given
expectThrows(InvalidIndexNameException.class, () -> resolveAbstractionsSelectorAllowed(List.of("<datetime-{now/M}>::custom")));
// == Single Patterned Index ==
// No selectors allowed, none given
assertThat(resolveAbstractionsSelectorNotAllowed(List.of("index*")), containsInAnyOrder("index1", "index2"));
// No selectors allowed, valid selector given
expectThrows(IllegalArgumentException.class, () -> resolveAbstractionsSelectorNotAllowed(List.of("index*::data")));
// Selectors allowed, valid selector given, data selector stripped off in result since it is the default
assertThat(resolveAbstractionsSelectorAllowed(List.of("index*::data")), containsInAnyOrder("index1", "index2"));
// Selectors allowed, wildcard selector provided, data selector stripped off in result since it is the default
// ** only returns ::data since expression is an index
assertThat(resolveAbstractionsSelectorAllowed(List.of("index*::*")), containsInAnyOrder("index1", "index2"));
// Selectors allowed, invalid selector given
expectThrows(InvalidIndexNameException.class, () -> resolveAbstractionsSelectorAllowed(List.of("index*::custom")));
// == Single Data Stream ==
// No selectors allowed, none given
assertThat(resolveAbstractionsSelectorNotAllowed(List.of("data-stream1")), contains("data-stream1"));
// No selectors allowed, valid selector given
expectThrows(IllegalArgumentException.class, () -> resolveAbstractionsSelectorNotAllowed(List.of("data-stream1::data")));
// Selectors allowed, valid selector given
assertThat(resolveAbstractionsSelectorAllowed(List.of("data-stream1::failures")), contains("data-stream1::failures"));
// Selectors allowed, wildcard selector provided
// ** returns both ::data and ::failures since expression is a data stream
// ** data selector stripped off in result since it is the default
assertThat(
resolveAbstractionsSelectorAllowed(List.of("data-stream1::*")),
containsInAnyOrder("data-stream1", "data-stream1::failures")
);
// Selectors allowed, invalid selector given
expectThrows(InvalidIndexNameException.class, () -> resolveAbstractionsSelectorAllowed(List.of("data-stream1::custom")));
// == Patterned Data Stream ==
// No selectors allowed, none given
assertThat(resolveAbstractionsSelectorNotAllowed(List.of("data-stream*")), contains("data-stream1"));
// No selectors allowed, valid selector given
expectThrows(IllegalArgumentException.class, () -> resolveAbstractionsSelectorNotAllowed(List.of("data-stream*::data")));
// Selectors allowed, valid selector given
assertThat(resolveAbstractionsSelectorAllowed(List.of("data-stream*::failures")), contains("data-stream1::failures"));
// Selectors allowed, wildcard selector provided
// ** returns both ::data and ::failures since expression is a data stream
assertThat(
resolveAbstractionsSelectorAllowed(List.of("data-stream*::*")),
containsInAnyOrder("data-stream1", "data-stream1::failures")
);
// Selectors allowed, invalid selector given
expectThrows(InvalidIndexNameException.class, () -> resolveAbstractionsSelectorAllowed(List.of("data-stream*::custom")));
// == Match All * Wildcard ==
// No selectors allowed, none given
assertThat(resolveAbstractionsSelectorNotAllowed(List.of("*")), containsInAnyOrder("index1", "index2", "data-stream1"));
// No selectors allowed, valid selector given
expectThrows(IllegalArgumentException.class, () -> resolveAbstractionsSelectorNotAllowed(List.of("*::data")));
// Selectors allowed, valid selector given
// ::data selector returns all values with data component
assertThat(resolveAbstractionsSelectorAllowed(List.of("*::data")), containsInAnyOrder("index1", "index2", "data-stream1"));
// Selectors allowed, valid selector given
// ::failures selector returns only data streams, which can have failure components
assertThat(resolveAbstractionsSelectorAllowed(List.of("*::failures")), contains("data-stream1::failures"));
// Selectors allowed, wildcard selector provided
// ** returns both ::data and ::failures for applicable abstractions
assertThat(
resolveAbstractionsSelectorAllowed(List.of("*::*")),
containsInAnyOrder("index1", "index2", "data-stream1", "data-stream1::failures")
);
// Selectors allowed, invalid selector given
expectThrows(InvalidIndexNameException.class, () -> resolveAbstractionsSelectorAllowed(List.of("*::custom")));
// == Wildcards with Exclusions ==
// No selectors allowed, none given
assertThat(resolveAbstractionsSelectorNotAllowed(List.of("*", "-index*")), containsInAnyOrder("data-stream1"));
// No selectors allowed, valid selector given
expectThrows(IllegalArgumentException.class, () -> resolveAbstractionsSelectorNotAllowed(List.of("*", "-*::data")));
// Selectors allowed, wildcard selector provided
// ** returns both ::data and ::failures for applicable abstractions
// ** limits the returned values based on selectors
assertThat(resolveAbstractionsSelectorAllowed(List.of("*::*", "-*::data")), contains("data-stream1::failures"));
// Selectors allowed, wildcard selector provided
// ** limits the returned values based on selectors
assertThat(
resolveAbstractionsSelectorAllowed(List.of("*::*", "-*::failures")),
containsInAnyOrder("index1", "index2", "data-stream1")
);
// Selectors allowed, none given, default to both selectors
// ** limits the returned values based on selectors
assertThat(
resolveAbstractionsSelectorAllowed(List.of("*", "-*::failures")),
containsInAnyOrder("index1", "index2", "data-stream1")
);
// Selectors allowed, invalid selector given
expectThrows(InvalidIndexNameException.class, () -> resolveAbstractionsSelectorAllowed(List.of("*", "-*::custom")));
}
public void testIsIndexVisible() {
assertThat(isIndexVisible("index1", null), is(true));
assertThat(isIndexVisible("index1", "*"), is(true));
assertThat(isIndexVisible("index1", "data"), is(true));
assertThat(isIndexVisible("index1", "failures"), is(false)); // *
// * Indices don't have failure components so the failure component is not visible
assertThat(isIndexVisible("data-stream1", null), is(true));
assertThat(isIndexVisible("data-stream1", "*"), is(true));
assertThat(isIndexVisible("data-stream1", "data"), is(true));
assertThat(isIndexVisible("data-stream1", "failures"), is(true));
}
private boolean isIndexVisible(String index, String selector) {
return IndexAbstractionResolver.isIndexVisible(
"*",
selector,
index,
IndicesOptions.strictExpandOpen(),
metadata,
indexNameExpressionResolver,
true
);
}
private List<String> resolveAbstractionsSelectorNotAllowed(List<String> expressions) {
return resolveAbstractions(expressions, IndicesOptions.strictExpandHiddenNoSelectors(), defaultMask);
}
private List<String> resolveAbstractionsSelectorAllowed(List<String> expressions) {
return resolveAbstractions(expressions, IndicesOptions.strictExpandOpen(), defaultMask);
}
private List<String> resolveAbstractions(List<String> expressions, IndicesOptions indicesOptions, Supplier<Set<String>> mask) {
return indexAbstractionResolver.resolveIndexAbstractions(expressions, indicesOptions, metadata, mask, (idx) -> true, true);
}
}

View file

@ -32,7 +32,6 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexVersion;
import org.elasticsearch.indices.FailureIndexNotSupportedException;
import org.elasticsearch.indices.IndexClosedException;
import org.elasticsearch.indices.InvalidIndexNameException;
import org.elasticsearch.indices.SystemIndexDescriptor;
@ -58,6 +57,8 @@ import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.elasticsearch.action.support.IndexComponentSelector.DATA;
import static org.elasticsearch.action.support.IndexComponentSelector.FAILURES;
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.backingIndexEqualTo;
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.createBackingIndex;
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.createFailureStore;
@ -1587,16 +1588,31 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
.put(indexBuilder("test-1").state(State.OPEN).putAlias(AliasMetadata.builder("alias-1")));
ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build();
assertEquals(resolvedExpressionsSet("alias-0", "alias-1"), indexNameExpressionResolver.resolveExpressions(state, "alias-*"));
assertEquals(
resolvedExpressionsSet("test-0", "alias-0", "alias-1"),
Set.of(new ResolvedExpression("alias-0", DATA), new ResolvedExpression("alias-1", DATA)),
indexNameExpressionResolver.resolveExpressions(state, "alias-*")
);
assertEquals(
Set.of(
new ResolvedExpression("test-0", DATA),
new ResolvedExpression("alias-0", DATA),
new ResolvedExpression("alias-1", DATA)
),
indexNameExpressionResolver.resolveExpressions(state, "test-0", "alias-*")
);
assertEquals(
resolvedExpressionsSet("test-0", "test-1", "alias-0", "alias-1"),
Set.of(
new ResolvedExpression("test-0", DATA),
new ResolvedExpression("test-1", DATA),
new ResolvedExpression("alias-0", DATA),
new ResolvedExpression("alias-1", DATA)
),
indexNameExpressionResolver.resolveExpressions(state, "test-*", "alias-*")
);
assertEquals(resolvedExpressionsSet("test-1", "alias-1"), indexNameExpressionResolver.resolveExpressions(state, "*-1"));
assertEquals(
Set.of(new ResolvedExpression("test-1", DATA), new ResolvedExpression("alias-1", DATA)),
indexNameExpressionResolver.resolveExpressions(state, "*-1")
);
}
public void testFilteringAliases() {
@ -2314,8 +2330,7 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
new IndicesOptions(
IndicesOptions.ConcreteTargetOptions.ERROR_WHEN_UNAVAILABLE_TARGETS,
IndicesOptions.WildcardOptions.DEFAULT,
IndicesOptions.GatekeeperOptions.builder().ignoreThrottled(true).build(),
IndicesOptions.SelectorOptions.DEFAULT
IndicesOptions.GatekeeperOptions.builder().ignoreThrottled(true).build()
),
"ind*",
"test-index"
@ -2750,12 +2765,10 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
assertThat(result[1].getName(), equalTo(DataStream.getDefaultBackingIndexName(dataStreamName, 2, epochMillis)));
}
// Test include failure store with an exact data stream name
// Test explicit include failure store with an exact data stream name
{
IndicesOptions indicesOptions = IndicesOptions.builder(IndicesOptions.STRICT_EXPAND_OPEN)
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.build();
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-data-stream");
IndicesOptions indicesOptions = IndicesOptions.STRICT_EXPAND_OPEN;
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-data-stream::*");
assertThat(result.length, equalTo(4));
assertThat(result[0].getName(), equalTo(DataStream.getDefaultBackingIndexName(dataStreamName, 1, epochMillis)));
assertThat(result[1].getName(), equalTo(DataStream.getDefaultBackingIndexName(dataStreamName, 2, epochMillis)));
@ -2763,43 +2776,22 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
assertThat(result[3].getName(), equalTo(DataStream.getDefaultFailureStoreName(dataStreamName, 2, epochMillis)));
}
// Test include failure store while we do not allow failure indices and ignore unavailable
// We expect that they will be skipped
// Test explicit include failure store while not allowing selectors
// We expect an error because selectors are disabled and one was provided
{
IndicesOptions indicesOptions = IndicesOptions.builder(IndicesOptions.STRICT_EXPAND_OPEN)
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.gatekeeperOptions(IndicesOptions.GatekeeperOptions.builder().allowFailureIndices(false).build())
.concreteTargetOptions(IndicesOptions.ConcreteTargetOptions.ALLOW_UNAVAILABLE_TARGETS)
.gatekeeperOptions(IndicesOptions.GatekeeperOptions.builder().allowSelectors(false).build())
.build();
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-data-stream");
assertThat(result.length, equalTo(2));
assertThat(result[0].getName(), equalTo(DataStream.getDefaultBackingIndexName(dataStreamName, 1, epochMillis)));
assertThat(result[1].getName(), equalTo(DataStream.getDefaultBackingIndexName(dataStreamName, 2, epochMillis)));
}
// Test include failure store while we do not allow failure indices
// We expect an error
{
IndicesOptions indicesOptions = IndicesOptions.builder(IndicesOptions.STRICT_EXPAND_OPEN)
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.gatekeeperOptions(IndicesOptions.GatekeeperOptions.builder().allowFailureIndices(false).build())
.build();
FailureIndexNotSupportedException failureIndexNotSupportedException = expectThrows(
FailureIndexNotSupportedException.class,
() -> indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-data-stream")
);
assertThat(
failureIndexNotSupportedException.getIndex().getName(),
equalTo(DataStream.getDefaultFailureStoreName(dataStreamName, 1, epochMillis))
expectThrows(
IllegalArgumentException.class,
() -> indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-data-stream::*")
);
}
// Test only failure store with an exact data stream name
// Test explicitly selecting only failure store with an exact data stream name
{
IndicesOptions indicesOptions = IndicesOptions.builder(IndicesOptions.STRICT_EXPAND_OPEN)
.selectorOptions(IndicesOptions.SelectorOptions.FAILURES)
.build();
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-data-stream");
IndicesOptions indicesOptions = IndicesOptions.STRICT_EXPAND_OPEN;
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-data-stream::failures");
assertThat(result.length, equalTo(2));
assertThat(result[0].getName(), equalTo(DataStream.getDefaultFailureStoreName(dataStreamName, 1, epochMillis)));
assertThat(result[1].getName(), equalTo(DataStream.getDefaultFailureStoreName(dataStreamName, 2, epochMillis)));
@ -2821,43 +2813,6 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
);
}
// Test include failure store without any expressions
{
IndicesOptions indicesOptions = IndicesOptions.builder(IndicesOptions.STRICT_EXPAND_OPEN)
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.build();
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true);
assertThat(result.length, equalTo(5));
List<String> indexNames = Arrays.stream(result).map(Index::getName).toList();
assertThat(
indexNames,
containsInAnyOrder(
DataStream.getDefaultBackingIndexName(dataStreamName, 2, epochMillis),
DataStream.getDefaultBackingIndexName(dataStreamName, 1, epochMillis),
DataStream.getDefaultFailureStoreName(dataStreamName, 2, epochMillis),
DataStream.getDefaultFailureStoreName(dataStreamName, 1, epochMillis),
otherIndex.getIndex().getName()
)
);
}
// Test only failure store without any expressions
{
IndicesOptions indicesOptions = IndicesOptions.builder(IndicesOptions.STRICT_EXPAND_OPEN)
.selectorOptions(IndicesOptions.SelectorOptions.FAILURES)
.build();
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true);
assertThat(result.length, equalTo(2));
List<String> indexNames = Arrays.stream(result).map(Index::getName).toList();
assertThat(
indexNames,
containsInAnyOrder(
DataStream.getDefaultFailureStoreName(dataStreamName, 2, epochMillis),
DataStream.getDefaultFailureStoreName(dataStreamName, 1, epochMillis)
)
);
}
// Test default with wildcard expression
{
IndicesOptions indicesOptions = IndicesOptions.STRICT_EXPAND_OPEN;
@ -2874,12 +2829,10 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
);
}
// Test include failure store with wildcard expression
// Test explicit include failure store with wildcard expression
{
IndicesOptions indicesOptions = IndicesOptions.builder(IndicesOptions.STRICT_EXPAND_OPEN)
.selectorOptions(IndicesOptions.SelectorOptions.ALL_APPLICABLE)
.build();
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-*");
IndicesOptions indicesOptions = IndicesOptions.STRICT_EXPAND_OPEN;
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-*::*");
assertThat(result.length, equalTo(5));
List<String> indexNames = Arrays.stream(result).map(Index::getName).toList();
assertThat(
@ -2894,12 +2847,10 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
);
}
// Test only failure store with wildcard expression
// Test explicit only failure store with wildcard expression
{
IndicesOptions indicesOptions = IndicesOptions.builder(IndicesOptions.STRICT_EXPAND_OPEN)
.selectorOptions(IndicesOptions.SelectorOptions.FAILURES)
.build();
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-*");
IndicesOptions indicesOptions = IndicesOptions.STRICT_EXPAND_OPEN;
Index[] result = indexNameExpressionResolver.concreteIndices(state, indicesOptions, true, "my-*::failures");
assertThat(result.length, equalTo(2));
List<String> indexNames = Arrays.stream(result).map(Index::getName).toList();
assertThat(
@ -3229,29 +3180,63 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
)
.build();
List<ResolvedExpression> streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.lenientExpand(), "log*");
List<String> names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), "log*");
assertEquals(Collections.singletonList(new ResolvedExpression(dataStream1, DATA)), streams);
assertEquals(Collections.singletonList(dataStream1), names);
streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.lenientExpand(), dataStream1);
names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), dataStream1);
assertEquals(Collections.singletonList(new ResolvedExpression(dataStream1, DATA)), streams);
assertEquals(Collections.singletonList(dataStream1), names);
streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.lenientExpand(), "other*");
names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), "other*");
assertEquals(Collections.singletonList(new ResolvedExpression(dataStream2, DATA)), streams);
assertEquals(Collections.singletonList(dataStream2), names);
streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.lenientExpand(), "*foobar");
names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), "*foobar");
assertThat(streams, containsInAnyOrder(new ResolvedExpression(dataStream1, DATA), new ResolvedExpression(dataStream2, DATA)));
assertThat(names, containsInAnyOrder(dataStream1, dataStream2));
streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.lenientExpand(), "notmatched");
names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), "notmatched");
assertThat(streams, empty());
assertThat(names, empty());
streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.lenientExpand(), index3.getIndex().getName());
names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), index3.getIndex().getName());
assertThat(streams, empty());
assertThat(names, empty());
streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.lenientExpand(), "*", "-logs-foobar");
names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), "*", "-logs-foobar");
assertThat(streams, containsInAnyOrder(new ResolvedExpression(dataStream2, DATA)));
assertThat(names, containsInAnyOrder(dataStream2));
streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.lenientExpand(), "*", "-*");
names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), "*", "-*");
assertThat(streams, empty());
assertThat(names, empty());
streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.strictExpandOpenAndForbidClosed(), "*foobar");
names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.strictExpandOpenAndForbidClosed(), "*foobar");
assertThat(streams, containsInAnyOrder(new ResolvedExpression(dataStream1, DATA), new ResolvedExpression(dataStream2, DATA)));
assertThat(names, containsInAnyOrder(dataStream1, dataStream2));
streams = indexNameExpressionResolver.dataStreams(state, IndicesOptions.lenientExpand(), "*foobar::*");
names = indexNameExpressionResolver.dataStreamNames(state, IndicesOptions.lenientExpand(), "*foobar::*");
assertThat(
streams,
containsInAnyOrder(
new ResolvedExpression(dataStream1, DATA),
new ResolvedExpression(dataStream1, FAILURES),
new ResolvedExpression(dataStream2, DATA),
new ResolvedExpression(dataStream2, FAILURES)
)
);
assertThat(names, containsInAnyOrder(dataStream1, dataStream2));
}
public void testDateMathMixedArray() {
@ -3285,10 +3270,10 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
assertThat(
result,
contains(
new ResolvedExpression("name1"),
new ResolvedExpression(dataMathIndex1),
new ResolvedExpression("name2"),
new ResolvedExpression(dateMathIndex2)
new ResolvedExpression("name1", DATA),
new ResolvedExpression(dataMathIndex1, DATA),
new ResolvedExpression("name2", DATA),
new ResolvedExpression(dateMathIndex2, DATA)
)
);
}

View file

@ -0,0 +1,183 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SelectorResolver;
import org.elasticsearch.core.Tuple;
import org.elasticsearch.indices.InvalidIndexNameException;
import org.elasticsearch.indices.SystemIndices;
import org.elasticsearch.test.ESTestCase;
import static org.elasticsearch.action.support.IndexComponentSelector.ALL_APPLICABLE;
import static org.elasticsearch.action.support.IndexComponentSelector.DATA;
import static org.elasticsearch.action.support.IndexComponentSelector.FAILURES;
import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.Context;
import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.nullValue;
import static org.mockito.Mockito.mock;
public class SelectorResolverTests extends ESTestCase {
public void testResolveExpression() {
// === Parsing and defaults
// Allow selectors TRUE
Context selectorsAllowed = getContext(getOptionsForSelectors());
assertThat(resolve(selectorsAllowed, "testXXX"), equalTo(new ResolvedExpression("testXXX", DATA)));
assertThat(resolve(selectorsAllowed, "testXXX::data"), equalTo(new ResolvedExpression("testXXX", DATA)));
assertThat(resolve(selectorsAllowed, "testXXX::failures"), equalTo(new ResolvedExpression("testXXX", FAILURES)));
assertThat(resolve(selectorsAllowed, "testXXX::*"), equalTo(new ResolvedExpression("testXXX", ALL_APPLICABLE)));
// Disallow selectors (example: creating, modifying, or deleting indices/data streams/aliases).
// Accepts standard expressions but throws when selectors are specified.
Context noSelectors = getContext(getDisabledSelectorOptions());
assertThat(resolve(noSelectors, "testXXX"), equalTo(new ResolvedExpression("testXXX")));
expectThrows(IllegalArgumentException.class, () -> resolve(noSelectors, "testXXX::data"));
expectThrows(IllegalArgumentException.class, () -> resolve(noSelectors, "testXXX::failures"));
expectThrows(IllegalArgumentException.class, () -> resolve(noSelectors, "testXXX::*"));
// === Errors
// Only recognized components can be selected
expectThrows(InvalidIndexNameException.class, () -> resolve(selectorsAllowed, "testXXX::custom"));
// Spelling is important
expectThrows(InvalidIndexNameException.class, () -> resolve(selectorsAllowed, "testXXX::failres"));
// Only the match all wildcard is supported
expectThrows(InvalidIndexNameException.class, () -> resolve(selectorsAllowed, "testXXX::d*ta"));
// Only one selector separator is allowed per expression
expectThrows(InvalidIndexNameException.class, () -> resolve(selectorsAllowed, "test::XXX::data"));
// === Wildcards, Date Math, and edge cases
// Wildcards are left as-is (handled in wildcard resolver)
assertThat(resolve(selectorsAllowed, "*"), equalTo(new ResolvedExpression("*", DATA)));
// Exclusions are left as-is (if no wildcards are present they are not treated as exclusions)
assertThat(resolve(selectorsAllowed, "-testXXX"), equalTo(new ResolvedExpression("-testXXX", DATA)));
// Exclusion syntax with selectors will have the selectors parsed
assertThat(resolve(selectorsAllowed, "-testXXX::failures"), equalTo(new ResolvedExpression("-testXXX", FAILURES)));
// Date math is left unprocessed (handled later in date math resolver)
assertThat(resolve(selectorsAllowed, "<test-{now/d}>"), equalTo(new ResolvedExpression("<test-{now/d}>", DATA)));
// Providing a selector requires adding after the date math brackets
assertThat(resolve(selectorsAllowed, "<test-{now/d}>::failures"), equalTo(new ResolvedExpression("<test-{now/d}>", FAILURES)));
// Selectors inside of date math expressions will trip an exception because they do not match an existing component name exactly
expectThrows(InvalidIndexNameException.class, () -> resolve(selectorsAllowed, "<test-{now/d}::failures>"));
// === Corner Cases
// Empty index name is not necessarily disallowed, but will be filtered out in the next steps of resolution
assertThat(resolve(selectorsAllowed, "::data"), equalTo(new ResolvedExpression("", DATA)));
// Remote cluster syntax is respected, even if code higher up the call stack is likely to already have handled it already
assertThat(resolve(selectorsAllowed, "cluster:index::data"), equalTo(new ResolvedExpression("cluster:index", DATA)));
// CCS with an empty index name is not necessarily disallowed, though other code in the resolution logic will likely throw
assertThat(resolve(selectorsAllowed, "cluster:::data"), equalTo(new ResolvedExpression("cluster:", DATA)));
// Same for empty cluster and index names
assertThat(resolve(selectorsAllowed, ":::data"), equalTo(new ResolvedExpression(":", DATA)));
// Any more prefix colon characters will trigger the multiple separators error logic
expectThrows(InvalidIndexNameException.class, () -> resolve(selectorsAllowed, "::::data"));
// Suffix case is not supported because there is no component named with the empty string
expectThrows(InvalidIndexNameException.class, () -> resolve(selectorsAllowed, "index::"));
}
public void testResolveMatchAllToSelectors() {
Context selectorsAllowed = getContext(getOptionsForSelectors());
assertThat(resolveMatchAllToSelector(selectorsAllowed, "*"), is(DATA));
assertThat(resolveMatchAllToSelector(selectorsAllowed, "*::data"), is(DATA));
assertThat(resolveMatchAllToSelector(selectorsAllowed, "*::failures"), is(FAILURES));
assertThat(resolveMatchAllToSelector(selectorsAllowed, "_all"), is(DATA));
assertThat(resolveMatchAllToSelector(selectorsAllowed, "_all::data"), is(DATA));
assertThat(resolveMatchAllToSelector(selectorsAllowed, "_all::failures"), is(FAILURES));
Context selectorsDisallowed = getContext(getDisabledSelectorOptions());
assertThat(resolveMatchAllToSelector(selectorsDisallowed, "*"), is(nullValue()));
expectThrows(IllegalArgumentException.class, () -> resolveMatchAllToSelector(selectorsDisallowed, "*::data"));
expectThrows(IllegalArgumentException.class, () -> resolveMatchAllToSelector(selectorsDisallowed, "*::failures"));
assertThat(resolveMatchAllToSelector(selectorsDisallowed, "_all"), is(nullValue()));
expectThrows(IllegalArgumentException.class, () -> resolveMatchAllToSelector(selectorsDisallowed, "_all::data"));
expectThrows(IllegalArgumentException.class, () -> resolveMatchAllToSelector(selectorsDisallowed, "_all::failures"));
}
public void testCombineExpressionWithSelector() {
expectThrows(NullPointerException.class, () -> IndexNameExpressionResolver.combineSelectorExpression(null, null));
expectThrows(NullPointerException.class, () -> IndexNameExpressionResolver.combineSelectorExpression(null, ""));
expectThrows(NullPointerException.class, () -> IndexNameExpressionResolver.combineSelectorExpression(null, "a"));
expectThrows(NullPointerException.class, () -> IndexNameExpressionResolver.combineSelectorExpression(null, "*"));
assertThat(IndexNameExpressionResolver.combineSelectorExpression("", null), is(equalTo("")));
assertThat(IndexNameExpressionResolver.combineSelectorExpression("", ""), is(equalTo("::")));
assertThat(IndexNameExpressionResolver.combineSelectorExpression("a", null), is(equalTo("a")));
assertThat(IndexNameExpressionResolver.combineSelectorExpression("a", ""), is(equalTo("a::")));
assertThat(IndexNameExpressionResolver.combineSelectorExpression("a", "b"), is(equalTo("a::b")));
assertThat(IndexNameExpressionResolver.combineSelectorExpression("a", "*"), is(equalTo("a::*")));
assertThat(IndexNameExpressionResolver.combineSelectorExpression("*", "b"), is(equalTo("*::b")));
assertThat(IndexNameExpressionResolver.combineSelectorExpression("*", "*"), is(equalTo("*::*")));
}
public void testHasSelectorSuffix() {
assertThat(IndexNameExpressionResolver.hasSelectorSuffix(null), is(false));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix(""), is(false));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("abcdefg"), is(false));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("*"), is(false));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("cluster:index"), is(false));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("index::data"), is(true));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("index::failures"), is(true));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("index::any"), is(true));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("index::*"), is(true));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("index::::::::::toomany"), is(true));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("cluster:index::data"), is(true));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("*:*::*"), is(true));
assertThat(IndexNameExpressionResolver.hasSelectorSuffix("index::value::value"), is(true));
}
public void testSplitSelectorExpression() {
expectThrows(NullPointerException.class, () -> IndexNameExpressionResolver.splitSelectorExpression(null));
assertThat(IndexNameExpressionResolver.splitSelectorExpression(""), is(equalTo(new Tuple<>("", null))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("a"), is(equalTo(new Tuple<>("a", null))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("*"), is(equalTo(new Tuple<>("*", null))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("index"), is(equalTo(new Tuple<>("index", null))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("cluster:index"), is(equalTo(new Tuple<>("cluster:index", null))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("*:index"), is(equalTo(new Tuple<>("*:index", null))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("cluster:*"), is(equalTo(new Tuple<>("cluster:*", null))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("*:*"), is(equalTo(new Tuple<>("*:*", null))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("*:*:*"), is(equalTo(new Tuple<>("*:*:*", null))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("a::data"), is(equalTo(new Tuple<>("a", "data"))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("a::failures"), is(equalTo(new Tuple<>("a", "failures"))));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("a::*"), is(equalTo(new Tuple<>("a", "*"))));
expectThrows(InvalidIndexNameException.class, () -> IndexNameExpressionResolver.splitSelectorExpression("a::random"));
expectThrows(InvalidIndexNameException.class, () -> IndexNameExpressionResolver.splitSelectorExpression("a::d*ta"));
expectThrows(InvalidIndexNameException.class, () -> IndexNameExpressionResolver.splitSelectorExpression("a::*ailures"));
expectThrows(InvalidIndexNameException.class, () -> IndexNameExpressionResolver.splitSelectorExpression("a::"));
expectThrows(InvalidIndexNameException.class, () -> IndexNameExpressionResolver.splitSelectorExpression("a::**"));
expectThrows(InvalidIndexNameException.class, () -> IndexNameExpressionResolver.splitSelectorExpression("index::data::*"));
assertThat(IndexNameExpressionResolver.splitSelectorExpression("::*"), is(equalTo(new Tuple<>("", "*"))));
}
private static IndicesOptions getOptionsForSelectors() {
return IndicesOptions.builder().gatekeeperOptions(IndicesOptions.GatekeeperOptions.builder().allowSelectors(true)).build();
}
private static IndicesOptions getDisabledSelectorOptions() {
return IndicesOptions.builder().gatekeeperOptions(IndicesOptions.GatekeeperOptions.builder().allowSelectors(false)).build();
}
private static Context getContext(IndicesOptions indicesOptions) {
return new Context(mock(ClusterState.class), indicesOptions, SystemIndices.SystemIndexAccessLevel.NONE);
}
private static ResolvedExpression resolve(Context context, String expression) {
return SelectorResolver.parseExpression(expression, context.getOptions());
}
private static IndexComponentSelector resolveMatchAllToSelector(Context context, String matchAll) {
return SelectorResolver.parseMatchAllToSelector(context, matchAll);
}
}

View file

@ -25,7 +25,11 @@ import java.util.Set;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import static org.elasticsearch.action.support.IndexComponentSelector.ALL_APPLICABLE;
import static org.elasticsearch.action.support.IndexComponentSelector.DATA;
import static org.elasticsearch.action.support.IndexComponentSelector.FAILURES;
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.createBackingIndex;
import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.createFailureStore;
import static org.elasticsearch.common.util.set.Sets.newHashSet;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.empty;
@ -50,19 +54,19 @@ public class WildcardExpressionResolverTests extends ESTestCase {
SystemIndexAccessLevel.NONE
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "ku*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "ku*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("kuku"))
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "test*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "test*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("testXXX", "testXYY", "testYYY"))
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "testX*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "testX*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("testXXX", "testXYY"))
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("testXXX", "testXYY", "testYYY", "kuku"))
);
}
@ -83,7 +87,7 @@ public class WildcardExpressionResolverTests extends ESTestCase {
SystemIndexAccessLevel.NONE
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "testX*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "testX*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("testXXX", "testXXY", "testXYY"))
);
context = new IndexNameExpressionResolver.Context(
@ -92,7 +96,7 @@ public class WildcardExpressionResolverTests extends ESTestCase {
SystemIndexAccessLevel.NONE
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "testX*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "testX*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("testXYY"))
);
context = new IndexNameExpressionResolver.Context(
@ -101,7 +105,7 @@ public class WildcardExpressionResolverTests extends ESTestCase {
SystemIndexAccessLevel.NONE
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "testX*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "testX*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("testXXX", "testXXY"))
);
}
@ -124,27 +128,31 @@ public class WildcardExpressionResolverTests extends ESTestCase {
SystemIndexAccessLevel.NONE
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "test*X*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "test*X*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("testXXX", "testXXY", "testXYY"))
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "test*X*Y")),
newHashSet(
IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "test*X*Y", ALL_APPLICABLE)
),
equalTo(resolvedExpressionsSet("testXXY", "testXYY"))
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "kuku*Y*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "kuku*Y*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("kukuYYY"))
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "*Y*")),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "*Y*", ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("testXXY", "testXYY", "testYYY", "kukuYYY"))
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "test*Y*X")).size(),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "test*Y*X", ALL_APPLICABLE))
.size(),
equalTo(0)
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "*Y*X")).size(),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, "*Y*X", ALL_APPLICABLE))
.size(),
equalTo(0)
);
}
@ -156,17 +164,37 @@ public class WildcardExpressionResolverTests extends ESTestCase {
.put(indexBuilder("testYYY"));
ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build();
{
IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(
state,
IndicesOptions.lenientExpandOpen(),
SystemIndexAccessLevel.NONE
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context)),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context, ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("testXXX", "testXYY", "testYYY"))
);
}
{
IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(
state,
IndicesOptions.builder(IndicesOptions.lenientExpandOpen())
.gatekeeperOptions(
IndicesOptions.GatekeeperOptions.builder(IndicesOptions.lenientExpandOpen().gatekeeperOptions())
.allowSelectors(false)
.build()
)
.build(),
SystemIndexAccessLevel.NONE
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context, null)),
equalTo(resolvedExpressionsNoSelectorSet("testXXX", "testXYY", "testYYY"))
);
}
}
public void testAllAliases() {
{
// hidden index with hidden alias should not be returned
@ -184,7 +212,10 @@ public class WildcardExpressionResolverTests extends ESTestCase {
IndicesOptions.lenientExpandOpen(), // don't include hidden
SystemIndexAccessLevel.NONE
);
assertThat(newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context)), equalTo(newHashSet()));
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context, ALL_APPLICABLE)),
equalTo(newHashSet())
);
}
{
@ -204,7 +235,7 @@ public class WildcardExpressionResolverTests extends ESTestCase {
SystemIndexAccessLevel.NONE
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context)),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context, ALL_APPLICABLE)),
equalTo(resolvedExpressionsSet("index-visible-alias"))
);
}
@ -215,6 +246,7 @@ public class WildcardExpressionResolverTests extends ESTestCase {
String dataStreamName = "foo_logs";
long epochMillis = randomLongBetween(1580536800000L, 1583042400000L);
IndexMetadata firstBackingIndexMetadata = createBackingIndex(dataStreamName, 1, epochMillis).build();
IndexMetadata firstFailureIndexMetadata = createFailureStore(dataStreamName, 1, epochMillis).build();
IndicesOptions indicesAndAliasesOptions = IndicesOptions.fromOptions(
randomBoolean(),
@ -231,7 +263,14 @@ public class WildcardExpressionResolverTests extends ESTestCase {
// visible data streams should be returned by _all even show backing indices are hidden
Metadata.Builder mdBuilder = Metadata.builder()
.put(firstBackingIndexMetadata, true)
.put(DataStreamTestHelper.newInstance(dataStreamName, List.of(firstBackingIndexMetadata.getIndex())));
.put(firstFailureIndexMetadata, true)
.put(
DataStreamTestHelper.newInstance(
dataStreamName,
List.of(firstBackingIndexMetadata.getIndex()),
List.of(firstFailureIndexMetadata.getIndex())
)
);
ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build();
@ -247,16 +286,33 @@ public class WildcardExpressionResolverTests extends ESTestCase {
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context)),
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context, DATA)),
equalTo(resolvedExpressionsSet(DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis)))
);
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context, ALL_APPLICABLE)),
equalTo(
resolvedExpressionsSet(
DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis)
)
)
);
}
{
// if data stream itself is hidden, backing indices should not be returned
var dataStream = DataStream.builder(dataStreamName, List.of(firstBackingIndexMetadata.getIndex())).setHidden(true).build();
var dataStream = DataStream.builder(dataStreamName, List.of(firstBackingIndexMetadata.getIndex()))
.setFailureIndices(
DataStream.DataStreamIndices.failureIndicesBuilder(List.of(firstFailureIndexMetadata.getIndex())).build()
)
.setHidden(true)
.build();
Metadata.Builder mdBuilder = Metadata.builder().put(firstBackingIndexMetadata, true).put(dataStream);
Metadata.Builder mdBuilder = Metadata.builder()
.put(firstBackingIndexMetadata, true)
.put(firstFailureIndexMetadata, true)
.put(dataStream);
ClusterState state = ClusterState.builder(new ClusterName("_name")).metadata(mdBuilder).build();
@ -271,7 +327,11 @@ public class WildcardExpressionResolverTests extends ESTestCase {
NONE
);
assertThat(newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context)), equalTo(newHashSet()));
assertThat(newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context, DATA)), equalTo(Set.of()));
assertThat(
newHashSet(IndexNameExpressionResolver.WildcardExpressionResolver.resolveAll(context, ALL_APPLICABLE)),
equalTo(Set.of())
);
}
}
@ -394,51 +454,57 @@ public class WildcardExpressionResolverTests extends ESTestCase {
{
Collection<ResolvedExpression> indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAndAliasesContext,
"foo_a*"
"foo_a*",
ALL_APPLICABLE
);
assertThat(indices, containsInAnyOrder(new ResolvedExpression("foo_index"), new ResolvedExpression("bar_index")));
assertThat(indices, containsInAnyOrder(new ResolvedExpression("foo_index", DATA), new ResolvedExpression("bar_index", DATA)));
}
{
Collection<ResolvedExpression> indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
skipAliasesLenientContext,
"foo_a*"
"foo_a*",
ALL_APPLICABLE
);
assertEquals(0, indices.size());
}
{
Set<ResolvedExpression> indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
skipAliasesStrictContext,
"foo_a*"
"foo_a*",
ALL_APPLICABLE
);
assertThat(indices, empty());
}
{
Collection<ResolvedExpression> indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAndAliasesContext,
"foo*"
"foo*",
ALL_APPLICABLE
);
assertThat(
indices,
containsInAnyOrder(
new ResolvedExpression("foo_foo"),
new ResolvedExpression("foo_index"),
new ResolvedExpression("bar_index")
new ResolvedExpression("foo_foo", DATA),
new ResolvedExpression("foo_index", DATA),
new ResolvedExpression("bar_index", DATA)
)
);
}
{
Collection<ResolvedExpression> indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
skipAliasesLenientContext,
"foo*"
"foo*",
ALL_APPLICABLE
);
assertThat(indices, containsInAnyOrder(new ResolvedExpression("foo_foo"), new ResolvedExpression("foo_index")));
assertThat(indices, containsInAnyOrder(new ResolvedExpression("foo_foo", DATA), new ResolvedExpression("foo_index", DATA)));
}
{
Collection<ResolvedExpression> indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
skipAliasesStrictContext,
"foo*"
"foo*",
ALL_APPLICABLE
);
assertThat(indices, containsInAnyOrder(new ResolvedExpression("foo_foo"), new ResolvedExpression("foo_index")));
assertThat(indices, containsInAnyOrder(new ResolvedExpression("foo_foo", DATA), new ResolvedExpression("foo_index", DATA)));
}
}
@ -447,6 +513,8 @@ public class WildcardExpressionResolverTests extends ESTestCase {
long epochMillis = randomLongBetween(1580536800000L, 1583042400000L);
IndexMetadata firstBackingIndexMetadata = createBackingIndex(dataStreamName, 1, epochMillis).build();
IndexMetadata secondBackingIndexMetadata = createBackingIndex(dataStreamName, 2, epochMillis).build();
IndexMetadata firstFailureIndexMetadata = createFailureStore(dataStreamName, 1, epochMillis).build();
IndexMetadata secondFailureIndexMetadata = createFailureStore(dataStreamName, 2, epochMillis).build();
Metadata.Builder mdBuilder = Metadata.builder()
.put(indexBuilder("foo_foo").state(State.OPEN))
@ -454,11 +522,14 @@ public class WildcardExpressionResolverTests extends ESTestCase {
.put(indexBuilder("foo_index").state(State.OPEN).putAlias(AliasMetadata.builder("foo_alias")))
.put(indexBuilder("bar_index").state(State.OPEN).putAlias(AliasMetadata.builder("foo_alias")))
.put(firstBackingIndexMetadata, true)
.put(firstFailureIndexMetadata, true)
.put(secondBackingIndexMetadata, true)
.put(secondFailureIndexMetadata, true)
.put(
DataStreamTestHelper.newInstance(
dataStreamName,
List.of(firstBackingIndexMetadata.getIndex(), secondBackingIndexMetadata.getIndex())
List.of(firstBackingIndexMetadata.getIndex(), secondBackingIndexMetadata.getIndex()),
List.of(firstFailureIndexMetadata.getIndex(), secondFailureIndexMetadata.getIndex())
)
);
@ -484,20 +555,25 @@ public class WildcardExpressionResolverTests extends ESTestCase {
// data streams are not included but expression matches the data stream
Collection<ResolvedExpression> indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAndAliasesContext,
"foo_*"
"foo_*",
ALL_APPLICABLE
);
assertThat(
indices,
containsInAnyOrder(
new ResolvedExpression("foo_index"),
new ResolvedExpression("foo_foo"),
new ResolvedExpression("bar_index")
new ResolvedExpression("foo_index", DATA),
new ResolvedExpression("foo_foo", DATA),
new ResolvedExpression("bar_index", DATA)
)
);
// data streams are not included and expression doesn't match the data steram
indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(indicesAndAliasesContext, "bar_*");
assertThat(indices, containsInAnyOrder(new ResolvedExpression("bar_bar"), new ResolvedExpression("bar_index")));
indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAndAliasesContext,
"bar_*",
ALL_APPLICABLE
);
assertThat(indices, containsInAnyOrder(new ResolvedExpression("bar_bar", DATA), new ResolvedExpression("bar_index", DATA)));
}
{
@ -525,33 +601,55 @@ public class WildcardExpressionResolverTests extends ESTestCase {
// data stream's corresponding backing indices are resolved
Collection<ResolvedExpression> indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAliasesAndDataStreamsContext,
"foo_*"
"foo_*",
ALL_APPLICABLE
);
assertThat(
indices,
containsInAnyOrder(
new ResolvedExpression("foo_index"),
new ResolvedExpression("bar_index"),
new ResolvedExpression("foo_foo"),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis)),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis))
new ResolvedExpression("foo_index", DATA),
new ResolvedExpression("bar_index", DATA),
new ResolvedExpression("foo_foo", DATA),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis), DATA),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis), DATA),
new ResolvedExpression(DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis), DATA),
new ResolvedExpression(DataStream.getDefaultFailureStoreName("foo_logs", 2, epochMillis), DATA)
)
);
// data stream's corresponding failure indices are resolved
indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAliasesAndDataStreamsContext,
"foo_*",
FAILURES
);
assertThat(
newHashSet(indices),
equalTo(
resolvedExpressionsSet(
DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 2, epochMillis)
)
)
);
// include all wildcard adds the data stream's backing indices
indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAliasesAndDataStreamsContext,
"*"
"*",
ALL_APPLICABLE
);
assertThat(
indices,
containsInAnyOrder(
new ResolvedExpression("foo_index"),
new ResolvedExpression("bar_index"),
new ResolvedExpression("foo_foo"),
new ResolvedExpression("bar_bar"),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis)),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis))
new ResolvedExpression("foo_index", DATA),
new ResolvedExpression("bar_index", DATA),
new ResolvedExpression("foo_foo", DATA),
new ResolvedExpression("bar_bar", DATA),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis), DATA),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis), DATA),
new ResolvedExpression(DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis), DATA),
new ResolvedExpression(DataStream.getDefaultFailureStoreName("foo_logs", 2, epochMillis), DATA)
)
);
}
@ -582,33 +680,116 @@ public class WildcardExpressionResolverTests extends ESTestCase {
// data stream's corresponding backing indices are resolved
Collection<ResolvedExpression> indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAliasesDataStreamsAndHiddenIndices,
"foo_*"
"foo_*",
ALL_APPLICABLE
);
assertThat(
indices,
containsInAnyOrder(
new ResolvedExpression("foo_index"),
new ResolvedExpression("bar_index"),
new ResolvedExpression("foo_foo"),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis)),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis))
new ResolvedExpression("foo_index", DATA),
new ResolvedExpression("bar_index", DATA),
new ResolvedExpression("foo_foo", DATA),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis), DATA),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis), DATA),
new ResolvedExpression(DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis), DATA),
new ResolvedExpression(DataStream.getDefaultFailureStoreName("foo_logs", 2, epochMillis), DATA)
)
);
// only data stream's corresponding failure indices are resolved
indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAliasesDataStreamsAndHiddenIndices,
"foo_*",
FAILURES
);
assertThat(
newHashSet(indices),
equalTo(
resolvedExpressionsSet(
DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 2, epochMillis)
)
)
);
// Resolve both backing and failure indices
indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAliasesDataStreamsAndHiddenIndices,
"foo_*",
ALL_APPLICABLE
);
assertThat(
newHashSet(indices),
equalTo(
resolvedExpressionsSet(
"foo_index",
"bar_index",
"foo_foo",
DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis),
DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 2, epochMillis)
)
)
);
// include all wildcard adds the data stream's backing indices
indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAliasesDataStreamsAndHiddenIndices,
"*"
"*",
ALL_APPLICABLE
);
assertThat(
indices,
containsInAnyOrder(
new ResolvedExpression("foo_index"),
new ResolvedExpression("bar_index"),
new ResolvedExpression("foo_foo"),
new ResolvedExpression("bar_bar"),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis)),
new ResolvedExpression(DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis))
newHashSet(indices),
equalTo(
resolvedExpressionsSet(
"foo_index",
"bar_index",
"foo_foo",
"bar_bar",
DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis),
DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 2, epochMillis)
)
)
);
// include all wildcard adds the data stream's failure indices
indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAliasesDataStreamsAndHiddenIndices,
"*",
FAILURES
);
assertThat(
newHashSet(indices),
equalTo(
resolvedExpressionsSet(
DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 2, epochMillis)
)
)
);
// include all wildcard adds the data stream's backing and failure indices
indices = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAliasesDataStreamsAndHiddenIndices,
"*",
ALL_APPLICABLE
);
assertThat(
newHashSet(indices),
equalTo(
resolvedExpressionsSet(
"foo_index",
"bar_index",
"foo_foo",
"bar_bar",
DataStream.getDefaultBackingIndexName("foo_logs", 1, epochMillis),
DataStream.getDefaultBackingIndexName("foo_logs", 2, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 1, epochMillis),
DataStream.getDefaultFailureStoreName("foo_logs", 2, epochMillis)
)
)
);
}
@ -642,34 +823,47 @@ public class WildcardExpressionResolverTests extends ESTestCase {
Collection<ResolvedExpression> matches = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAndAliasesContext,
"*"
"*",
ALL_APPLICABLE
);
assertThat(
matches,
containsInAnyOrder(
new ResolvedExpression("bar_bar"),
new ResolvedExpression("foo_foo"),
new ResolvedExpression("foo_index"),
new ResolvedExpression("bar_index")
new ResolvedExpression("bar_bar", DATA),
new ResolvedExpression("foo_foo", DATA),
new ResolvedExpression("foo_index", DATA),
new ResolvedExpression("bar_index", DATA)
)
);
matches = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(onlyIndicesContext, "*");
matches = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(onlyIndicesContext, "*", ALL_APPLICABLE);
assertThat(
matches,
containsInAnyOrder(
new ResolvedExpression("bar_bar"),
new ResolvedExpression("foo_foo"),
new ResolvedExpression("foo_index"),
new ResolvedExpression("bar_index")
new ResolvedExpression("bar_bar", DATA),
new ResolvedExpression("foo_foo", DATA),
new ResolvedExpression("foo_index", DATA),
new ResolvedExpression("bar_index", DATA)
)
);
matches = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(indicesAndAliasesContext, "foo*");
matches = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
indicesAndAliasesContext,
"foo*",
ALL_APPLICABLE
);
assertThat(
matches,
containsInAnyOrder(new ResolvedExpression("foo_foo"), new ResolvedExpression("foo_index"), new ResolvedExpression("bar_index"))
containsInAnyOrder(
new ResolvedExpression("foo_foo", DATA),
new ResolvedExpression("foo_index", DATA),
new ResolvedExpression("bar_index", DATA)
)
);
matches = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(onlyIndicesContext, "foo*");
assertThat(matches, containsInAnyOrder(new ResolvedExpression("foo_foo"), new ResolvedExpression("foo_index")));
matches = IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(
onlyIndicesContext,
"foo*",
ALL_APPLICABLE
);
assertThat(matches, containsInAnyOrder(new ResolvedExpression("foo_foo", DATA), new ResolvedExpression("foo_index", DATA)));
}
private static IndexMetadata.Builder indexBuilder(String index, boolean hidden) {
@ -682,10 +876,17 @@ public class WildcardExpressionResolverTests extends ESTestCase {
}
private static void assertWildcardResolvesToEmpty(IndexNameExpressionResolver.Context context, String wildcardExpression) {
assertThat(IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, wildcardExpression), empty());
assertThat(
IndexNameExpressionResolver.WildcardExpressionResolver.matchWildcardToResources(context, wildcardExpression, DATA),
empty()
);
}
private Set<ResolvedExpression> resolvedExpressionsSet(String... expressions) {
return Arrays.stream(expressions).map(ResolvedExpression::new).collect(Collectors.toSet());
return Arrays.stream(expressions).map(e -> new ResolvedExpression(e, DATA)).collect(Collectors.toSet());
}
private Set<ResolvedExpression> resolvedExpressionsNoSelectorSet(String... expressions) {
return Arrays.stream(expressions).map(e -> new ResolvedExpression(e)).collect(Collectors.toSet());
}
}

View file

@ -77,7 +77,7 @@ public class DataStreamRestIT extends ESRestTestCase {
indexRequest.setJsonEntity("{\"@timestamp\": \"2020-01-01\"}");
client().performRequest(indexRequest);
// Initialize the failure store
rollover = new Request("POST", "/fs/_rollover?target_failure_store=true");
rollover = new Request("POST", "/fs::failures/_rollover");
client().performRequest(rollover);
dataStreams = (Map<?, ?>) getLocation("/_xpack/usage").get("data_streams");

View file

@ -124,7 +124,7 @@ public class LifecyclePolicyUtils {
final List<String> allDataStreams = indexNameExpressionResolver.dataStreamNames(
state,
IndicesOptions.LENIENT_EXPAND_OPEN_CLOSED_HIDDEN
IndicesOptions.LENIENT_EXPAND_OPEN_CLOSED_HIDDEN_NO_SELECTOR
);
final List<String> dataStreams = allDataStreams.stream().filter(dsName -> {

View file

@ -11,13 +11,14 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.Strings;
import org.elasticsearch.core.TimeValue;
@ -122,9 +123,7 @@ public class RolloverStep extends AsyncActionStep {
// Calling rollover with no conditions will always roll over the index
RolloverRequest rolloverRequest = new RolloverRequest(rolloverTarget, null).masterNodeTimeout(TimeValue.MAX_VALUE);
if (targetFailureStore) {
rolloverRequest.setIndicesOptions(
IndicesOptions.builder(rolloverRequest.indicesOptions()).selectorOptions(IndicesOptions.SelectorOptions.FAILURES).build()
);
rolloverRequest.setRolloverTarget(IndexNameExpressionResolver.combineSelector(rolloverTarget, IndexComponentSelector.FAILURES));
}
// We don't wait for active shards when we perform the rollover because the
// {@link org.elasticsearch.xpack.core.ilm.WaitForActiveShardsStep} step will do so

View file

@ -12,11 +12,12 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.rollover.RolloverConditions;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.unit.ByteSizeValue;
@ -242,9 +243,7 @@ public class WaitForRolloverReadyStep extends AsyncWaitStep {
rolloverRequest.dryRun(true);
rolloverRequest.setConditions(applyDefaultConditions(conditions, rolloverOnlyIfHasDocuments));
if (targetFailureStore) {
rolloverRequest.setIndicesOptions(
IndicesOptions.builder(rolloverRequest.indicesOptions()).selectorOptions(IndicesOptions.SelectorOptions.FAILURES).build()
);
rolloverRequest.setRolloverTarget(IndexNameExpressionResolver.combineSelector(rolloverTarget, IndexComponentSelector.FAILURES));
}
return rolloverRequest;
}

View file

@ -6,11 +6,13 @@
*/
package org.elasticsearch.xpack.core.security.authz.accesscontrol;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.util.CachedSupplier;
import org.elasticsearch.common.util.Maps;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.Tuple;
import org.elasticsearch.xpack.core.security.authz.IndicesAndAliasesResolverField;
import org.elasticsearch.xpack.core.security.authz.permission.DocumentPermissions;
import org.elasticsearch.xpack.core.security.authz.permission.FieldPermissions;
@ -59,7 +61,8 @@ public class IndicesAccessControl {
*/
@Nullable
public IndexAccessControl getIndexPermissions(String index) {
return this.getAllIndexPermissions().get(index);
Tuple<String, String> indexAndSelector = IndexNameExpressionResolver.splitSelectorExpression(index);
return this.getAllIndexPermissions().get(indexAndSelector.v1());
}
public boolean hasIndexPermissions(String index) {

View file

@ -10,8 +10,10 @@ import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.Operations;
import org.elasticsearch.action.admin.indices.mapping.put.TransportAutoPutMappingAction;
import org.elasticsearch.action.admin.indices.mapping.put.TransportPutMappingAction;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.logging.DeprecationCategory;
@ -354,18 +356,34 @@ public final class IndicesPermission {
*/
private final String name;
/**
* The selector to be applied to the IndexAbstraction which selects which indices to return when resolving
*/
@Nullable
private final IndexComponentSelector selector;
/**
* The IndexAbstraction on which authorization is being performed, or {@code null} if nothing in the cluster matches the name
*/
@Nullable
private final IndexAbstraction indexAbstraction;
private IndexResource(String name, @Nullable IndexAbstraction abstraction) {
private IndexResource(String name, @Nullable IndexAbstraction abstraction, @Nullable IndexComponentSelector selector) {
assert name != null : "Resource name cannot be null";
assert abstraction == null || abstraction.getName().equals(name)
: "Index abstraction has unexpected name [" + abstraction.getName() + "] vs [" + name + "]";
assert abstraction == null
|| selector == null
|| IndexComponentSelector.FAILURES.equals(selector) == false
|| abstraction.isDataStreamRelated()
: "Invalid index component selector ["
+ selector.getKey()
+ "] applied to abstraction of type ["
+ abstraction.getType()
+ "]";
this.name = name;
this.indexAbstraction = abstraction;
this.selector = selector;
}
/**
@ -402,21 +420,69 @@ public final class IndicesPermission {
/**
* @return the number of distinct objects to which this expansion refers.
*/
public int size() {
public int size(Map<String, IndexAbstraction> lookup) {
if (indexAbstraction == null) {
return 1;
} else if (indexAbstraction.getType() == IndexAbstraction.Type.CONCRETE_INDEX) {
return 1;
} else if (selector != null) {
int size = 1;
if (selector.shouldIncludeData()) {
size += indexAbstraction.getIndices().size();
}
if (selector.shouldIncludeFailures()) {
if (IndexAbstraction.Type.ALIAS.equals(indexAbstraction.getType())) {
Set<DataStream> aliasDataStreams = new HashSet<>();
int failureIndices = 0;
for (Index index : indexAbstraction.getIndices()) {
DataStream parentDataStream = lookup.get(index.getName()).getParentDataStream();
if (parentDataStream != null && aliasDataStreams.add(parentDataStream)) {
failureIndices += parentDataStream.getFailureIndices().getIndices().size();
}
}
size += failureIndices;
} else {
DataStream parentDataStream = (DataStream) indexAbstraction;
size += parentDataStream.getFailureIndices().getIndices().size();
}
}
return size;
} else {
return 1 + indexAbstraction.getIndices().size();
}
}
public Collection<String> resolveConcreteIndices() {
public Collection<String> resolveConcreteIndices(Map<String, IndexAbstraction> lookup) {
if (indexAbstraction == null) {
return List.of();
} else if (indexAbstraction.getType() == IndexAbstraction.Type.CONCRETE_INDEX) {
return List.of(indexAbstraction.getName());
} else if (IndexComponentSelector.FAILURES.equals(selector)) {
if (IndexAbstraction.Type.ALIAS.equals(indexAbstraction.getType())) {
Set<DataStream> aliasDataStreams = new HashSet<>();
for (Index index : indexAbstraction.getIndices()) {
DataStream parentDataStream = lookup.get(index.getName()).getParentDataStream();
if (parentDataStream != null) {
aliasDataStreams.add(parentDataStream);
}
}
List<String> concreteIndexNames = new ArrayList<>(aliasDataStreams.size());
for (DataStream aliasDataStream : aliasDataStreams) {
DataStream.DataStreamIndices failureIndices = aliasDataStream.getFailureIndices();
for (Index index : failureIndices.getIndices()) {
concreteIndexNames.add(index.getName());
}
}
return concreteIndexNames;
} else {
DataStream parentDataStream = (DataStream) indexAbstraction;
DataStream.DataStreamIndices failureIndices = parentDataStream.getFailureIndices();
List<String> concreteIndexNames = new ArrayList<>(failureIndices.getIndices().size());
for (Index index : failureIndices.getIndices()) {
concreteIndexNames.add(index.getName());
}
return concreteIndexNames;
}
} else {
final List<Index> indices = indexAbstraction.getIndices();
final List<String> concreteIndexNames = new ArrayList<>(indices.size());
@ -452,9 +518,15 @@ public final class IndicesPermission {
int totalResourceCount = 0;
for (String indexOrAlias : requestedIndicesOrAliases) {
final IndexResource resource = new IndexResource(indexOrAlias, lookup.get(indexOrAlias));
// Remove any selectors from abstraction name. Discard them for this check as we do not have access control for them (yet)
Tuple<String, String> expressionAndSelector = IndexNameExpressionResolver.splitSelectorExpression(indexOrAlias);
indexOrAlias = expressionAndSelector.v1();
IndexComponentSelector selector = expressionAndSelector.v2() == null
? null
: IndexComponentSelector.getByKey(expressionAndSelector.v2());
final IndexResource resource = new IndexResource(indexOrAlias, lookup.get(indexOrAlias), selector);
resources.put(resource.name, resource);
totalResourceCount += resource.size();
totalResourceCount += resource.size(lookup);
}
final boolean overallGranted = isActionGranted(action, resources);
@ -463,7 +535,8 @@ public final class IndicesPermission {
action,
resources,
finalTotalResourceCount,
fieldPermissionsCache
fieldPermissionsCache,
lookup
);
return new IndicesAccessControl(overallGranted, indexPermissions);
@ -473,7 +546,8 @@ public final class IndicesPermission {
final String action,
final Map<String, IndexResource> requestedResources,
final int totalResourceCount,
final FieldPermissionsCache fieldPermissionsCache
final FieldPermissionsCache fieldPermissionsCache,
final Map<String, IndexAbstraction> lookup
) {
// now... every index that is associated with the request, must be granted
@ -488,7 +562,7 @@ public final class IndicesPermission {
// true if ANY group covers the given index AND the given action
boolean granted = false;
final Collection<String> concreteIndices = resource.resolveConcreteIndices();
final Collection<String> concreteIndices = resource.resolveConcreteIndices(lookup);
for (Group group : groups) {
// the group covers the given index OR the given index is a backing index and the group covers the parent data stream
if (resource.checkIndex(group)) {

View file

@ -11,11 +11,13 @@ import org.elasticsearch.action.admin.indices.rollover.MaxSizeCondition;
import org.elasticsearch.action.admin.indices.rollover.RolloverInfo;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverResponse;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.AliasMetadata;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.IndexVersion;
@ -68,11 +70,15 @@ public class RolloverStepTests extends AbstractStepTestCase<RolloverStep> {
.build();
}
private static void assertRolloverIndexRequest(RolloverRequest request, String rolloverTarget) {
private static void assertRolloverIndexRequest(RolloverRequest request, String rolloverTarget, boolean targetFailureStores) {
String target = targetFailureStores
? IndexNameExpressionResolver.combineSelector(rolloverTarget, IndexComponentSelector.FAILURES)
: rolloverTarget;
assertNotNull(request);
assertEquals(1, request.indices().length);
assertEquals(rolloverTarget, request.indices()[0]);
assertEquals(rolloverTarget, request.getRolloverTarget());
assertEquals(target, request.indices()[0]);
assertEquals(target, request.getRolloverTarget());
assertFalse(request.isDryRun());
assertEquals(0, request.getConditions().getConditions().size());
}
@ -83,7 +89,7 @@ public class RolloverStepTests extends AbstractStepTestCase<RolloverStep> {
RolloverStep step = createRandomInstance();
mockClientRolloverCall(alias);
mockClientRolloverCall(alias, false);
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).metadata(Metadata.builder().put(indexMetadata, true)).build();
performActionAndWait(step, indexMetadata, clusterState, null);
@ -109,8 +115,6 @@ public class RolloverStepTests extends AbstractStepTestCase<RolloverStep> {
RolloverStep step = createRandomInstance();
mockClientRolloverCall(dataStreamName);
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metadata(
Metadata.builder()
@ -121,6 +125,7 @@ public class RolloverStepTests extends AbstractStepTestCase<RolloverStep> {
.build();
boolean useFailureStore = randomBoolean();
IndexMetadata indexToOperateOn = useFailureStore ? failureIndexMetadata : indexMetadata;
mockClientRolloverCall(dataStreamName, useFailureStore);
performActionAndWait(step, indexToOperateOn, clusterState, null);
Mockito.verify(client, Mockito.only()).admin();
@ -179,12 +184,12 @@ public class RolloverStepTests extends AbstractStepTestCase<RolloverStep> {
verifyNoMoreInteractions(indicesClient);
}
private void mockClientRolloverCall(String rolloverTarget) {
private void mockClientRolloverCall(String rolloverTarget, boolean targetFailureStore) {
Mockito.doAnswer(invocation -> {
RolloverRequest request = (RolloverRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<RolloverResponse> listener = (ActionListener<RolloverResponse>) invocation.getArguments()[1];
assertRolloverIndexRequest(request, rolloverTarget);
assertRolloverIndexRequest(request, rolloverTarget, targetFailureStore);
listener.onResponse(new RolloverResponse(null, null, Map.of(), request.isDryRun(), true, true, true, false));
return null;
}).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any());
@ -237,7 +242,7 @@ public class RolloverStepTests extends AbstractStepTestCase<RolloverStep> {
RolloverRequest request = (RolloverRequest) invocation.getArguments()[0];
@SuppressWarnings("unchecked")
ActionListener<RolloverResponse> listener = (ActionListener<RolloverResponse>) invocation.getArguments()[1];
assertRolloverIndexRequest(request, alias);
assertRolloverIndexRequest(request, alias, false);
listener.onFailure(exception);
return null;
}).when(indicesClient).rolloverIndex(Mockito.any(), Mockito.any());

View file

@ -24,6 +24,7 @@ import org.elasticsearch.action.admin.indices.rollover.RolloverConditions;
import org.elasticsearch.action.admin.indices.rollover.RolloverInfo;
import org.elasticsearch.action.admin.indices.rollover.RolloverRequest;
import org.elasticsearch.action.admin.indices.rollover.RolloverResponse;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.cluster.metadata.AliasMetadata;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
@ -45,9 +46,9 @@ import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SelectorResolver.SELECTOR_SEPARATOR;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
@ -270,7 +271,11 @@ public class WaitForRolloverReadyStepTests extends AbstractStepTestCase<WaitForR
WaitForRolloverReadyStep step = createRandomInstance();
mockRolloverIndexCall(dataStreamName, step, true);
mockRolloverIndexCall(
failureStoreIndex ? dataStreamName + SELECTOR_SEPARATOR + IndexComponentSelector.FAILURES.getKey() : dataStreamName,
step,
true
);
SetOnce<Boolean> conditionsMet = new SetOnce<>();
Metadata metadata = Metadata.builder()
@ -307,8 +312,14 @@ public class WaitForRolloverReadyStepTests extends AbstractStepTestCase<WaitForR
verify(indicesClient, Mockito.only()).rolloverIndex(requestCaptor.capture(), Mockito.any());
RolloverRequest request = requestCaptor.getValue();
assertThat(request.indicesOptions().includeFailureIndices(), equalTo(failureStoreIndex));
assertThat(request.indicesOptions().includeRegularIndices(), not(equalTo(failureStoreIndex)));
if (failureStoreIndex == false) {
assertThat(request.getRolloverTarget(), equalTo(dataStreamName));
} else {
assertThat(
request.getRolloverTarget(),
equalTo(dataStreamName + SELECTOR_SEPARATOR + IndexComponentSelector.FAILURES.getKey())
);
}
}
public void testSkipRolloverIfDataStreamIsAlreadyRolledOver() {

View file

@ -304,7 +304,7 @@ public class DeprecationInfoAction extends ActionType<DeprecationInfoAction.Resp
List<String> dataStreamNames = indexNameExpressionResolver.dataStreamNames(
state,
IndicesOptions.LENIENT_EXPAND_OPEN_CLOSED_HIDDEN
IndicesOptions.LENIENT_EXPAND_OPEN_CLOSED_HIDDEN_NO_SELECTOR
);
Map<String, List<DeprecationIssue>> dataStreamIssues = new HashMap<>();
for (String dataStreamName : dataStreamNames) {

View file

@ -82,7 +82,7 @@ public class AnalyticsCollectionResolver {
// Listing data streams that are matching the analytics collection pattern.
List<String> dataStreams = indexNameExpressionResolver.dataStreamNames(
state,
IndicesOptions.lenientExpandOpen(),
IndicesOptions.lenientExpandOpenNoSelectors(),
EVENT_DATA_STREAM_INDEX_PATTERN
);

View file

@ -348,8 +348,8 @@ public class DatafeedNodeSelectorTests extends ESTestCase {
"cannot start datafeed [datafeed_id] because it failed resolving indices given [not_foo] and "
+ "indices_options [IndicesOptions[ignore_unavailable=false, allow_no_indices=true, expand_wildcards_open=true, "
+ "expand_wildcards_closed=false, expand_wildcards_hidden=false, allow_aliases_to_multiple_indices=true, "
+ "forbid_closed_indices=true, ignore_aliases=false, ignore_throttled=true, include_regular_indices=true, "
+ "include_failure_indices=false, allow_failure_indices=true]] with exception [no such index [not_foo]]"
+ "forbid_closed_indices=true, ignore_aliases=false, ignore_throttled=true, "
+ "allow_selectors=true]] with exception [no such index [not_foo]]"
)
)
);
@ -383,8 +383,7 @@ public class DatafeedNodeSelectorTests extends ESTestCase {
+ "indices given [not_foo] and indices_options [IndicesOptions[ignore_unavailable=false, allow_no_indices=true, "
+ "expand_wildcards_open=true, expand_wildcards_closed=false, expand_wildcards_hidden=false, "
+ "allow_aliases_to_multiple_indices=true, forbid_closed_indices=true, ignore_aliases=false, "
+ "ignore_throttled=true, include_regular_indices=true, include_failure_indices=false, "
+ "allow_failure_indices=true]] with exception [no such index [not_foo]]]"
+ "ignore_throttled=true, allow_selectors=true]] with exception [no such index [not_foo]]]"
)
)
);
@ -560,8 +559,8 @@ public class DatafeedNodeSelectorTests extends ESTestCase {
+ "[cannot start datafeed [datafeed_id] because it failed resolving indices given [not_foo] and "
+ "indices_options [IndicesOptions[ignore_unavailable=false, allow_no_indices=true, expand_wildcards_open=true, "
+ "expand_wildcards_closed=false, expand_wildcards_hidden=false, allow_aliases_to_multiple_indices=true, "
+ "forbid_closed_indices=true, ignore_aliases=false, ignore_throttled=true, include_regular_indices=true, "
+ "include_failure_indices=false, allow_failure_indices=true]] with exception [no such index [not_foo]]]"
+ "forbid_closed_indices=true, ignore_aliases=false, ignore_throttled=true, "
+ "allow_selectors=true]] with exception [no such index [not_foo]]]"
)
)
);

View file

@ -13,6 +13,7 @@ import org.elasticsearch.action.admin.indices.alias.get.GetAliasesRequest;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
import org.elasticsearch.action.search.SearchContextId;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.IndexComponentSelector;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.metadata.AliasMetadata;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
@ -25,6 +26,7 @@ import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.Tuple;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.transport.NoSuchRemoteClusterException;
@ -90,6 +92,15 @@ class IndicesAndAliasesResolver {
* be added to the <em>remote</em> index list.
* <br>
* Otherwise, <em>N</em> will be added to the <em>local</em> index list.
* <br><br>
* <p>
* If the provided <code>request</code> is of a type that {@link IndicesOptions#allowSelectors() allows selectors}, then the given index
* names may be modified to include any {@link IndexComponentSelector selectors} that should be
* returned. If a wildcard selector is present on the index name then the wildcard will be resolved to
* {@link IndexComponentSelector#values() all applicable concrete selector names} for a given index abstraction. Selectors that are
* not applicable for an index abstraction (such as the <code>::failures</code> selector on non-data streams) are not returned when
* resolving wildcards.
* </p>
*/
ResolvedIndices resolve(
@ -151,7 +162,12 @@ class IndicesAndAliasesResolver {
if (indices == null || indices.length == 0) {
throw new IllegalArgumentException("the action " + action + " requires explicit index names, but none were provided");
}
if (IndexNameExpressionResolver.isAllIndices(Arrays.asList(indices))) {
// We need to parse off any potential selector suffixes to see if the index part of an expression is a match all
// Do so lazily to avoid extra work in case of a large number of indices
if (IndexNameExpressionResolver.isAllIndices(
Arrays.asList(indices),
(expr) -> IndexNameExpressionResolver.splitSelectorExpression(expr).v1()
)) {
throw new IllegalArgumentException(
"the action "
+ action
@ -186,11 +202,25 @@ class IndicesAndAliasesResolver {
// shard level requests.
final List<String> localIndices = new ArrayList<>(split.getLocal().size());
for (String localName : split.getLocal()) {
// TODO: Shard level requests should already have their selectors resolved to concrete indices by their parent requests
// see https://github.com/elastic/elasticsearch/issues/119629
// Resolve them here as long as they are non-wildcard, or missing from the expression
Tuple<String, String> expressionTuple = IndexNameExpressionResolver.splitSelectorExpression(localName);
String selector = expressionTuple.v2();
if (selector != null && Regex.isSimpleMatchPattern(selector)) {
throwOnUnexpectedWildcardSelectors(action, split.getLocal());
}
String localExpression = expressionTuple.v1();
// TODO: Shard level requests have wildcard expanded already and do not need go through this check
if (Regex.isSimpleMatchPattern(localName)) {
// see https://github.com/elastic/elasticsearch/issues/119629
if (Regex.isSimpleMatchPattern(localExpression)) {
throwOnUnexpectedWildcards(action, split.getLocal());
}
localIndices.add(IndexNameExpressionResolver.resolveDateMathExpression(localName));
String finalExpression = IndexNameExpressionResolver.resolveDateMathExpression(localExpression);
if (selector != null) {
finalExpression = IndexNameExpressionResolver.combineSelectorExpression(finalExpression, selector);
}
localIndices.add(finalExpression);
}
return new ResolvedIndices(localIndices, split.getRemote());
@ -227,6 +257,19 @@ class IndicesAndAliasesResolver {
);
}
private static void throwOnUnexpectedWildcardSelectors(String action, List<String> indices) {
final List<String> selectorExpressions = indices.stream().filter(IndexNameExpressionResolver::hasSelectorSuffix).toList();
assert selectorExpressions.isEmpty() == false : "we already know that there's at least one selector in the indices";
throw new IllegalArgumentException(
"the action "
+ action
+ " does not support wildcard selectors;"
+ " the provided index expression(s) ["
+ Strings.collectionToCommaDelimitedString(selectorExpressions)
+ "] are not allowed"
);
}
ResolvedIndices resolveIndicesAndAliases(
String action,
IndicesRequest indicesRequest,
@ -250,18 +293,48 @@ class IndicesAndAliasesResolver {
final IndicesOptions indicesOptions = indicesRequest.indicesOptions();
// check for all and return list of authorized indices
if (IndexNameExpressionResolver.isAllIndices(indicesList(indicesRequest.indices()))) {
boolean isAllIndices;
String allIndicesPatternSelector = null;
if (indicesRequest.indices() != null && indicesRequest.indices().length > 0) {
// Always parse selectors, but do so lazily so that we don't spend a lot of time splitting strings each resolution
isAllIndices = IndexNameExpressionResolver.isAllIndices(
indicesList(indicesRequest.indices()),
(expr) -> IndexNameExpressionResolver.splitSelectorExpression(expr).v1()
);
if (isAllIndices) {
// This parses the single all-indices expression for a second time in this conditional branch, but this is better than
// parsing a potentially big list of indices on every request.
allIndicesPatternSelector = IndexNameExpressionResolver.splitSelectorExpression(
indicesList(indicesRequest.indices()).getFirst()
).v2();
}
} else {
isAllIndices = IndexNameExpressionResolver.isAllIndices(indicesList(indicesRequest.indices()));
}
if (isAllIndices) {
// First, if a selector is present, check to make sure that selectors are even allowed here
if (indicesOptions.allowSelectors() == false && allIndicesPatternSelector != null) {
String originalIndexExpression = indicesRequest.indices()[0];
throw new IllegalArgumentException(
"Index component selectors are not supported in this context but found selector in expression ["
+ originalIndexExpression
+ "]"
);
}
if (indicesOptions.expandWildcardExpressions()) {
for (String authorizedIndex : authorizedIndices.all().get()) {
if (IndexAbstractionResolver.isIndexVisible(
"*",
allIndicesPatternSelector,
authorizedIndex,
indicesOptions,
metadata,
nameExpressionResolver,
indicesRequest.includeDataStreams()
)) {
resolvedIndicesBuilder.addLocal(authorizedIndex);
resolvedIndicesBuilder.addLocal(
IndexNameExpressionResolver.combineSelectorExpression(authorizedIndex, allIndicesPatternSelector)
);
}
}
}

View file

@ -32,6 +32,7 @@ import org.elasticsearch.action.search.TransportClosePointInTimeAction;
import org.elasticsearch.action.search.TransportMultiSearchAction;
import org.elasticsearch.action.search.TransportSearchScrollAction;
import org.elasticsearch.action.termvectors.MultiTermVectorsAction;
import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
@ -880,6 +881,10 @@ public class RBACEngine implements AuthorizationEngine {
for (Index index : indexAbstraction.getIndices()) {
indicesAndAliases.add(index.getName());
}
// TODO: We need to limit if a data stream's failure indices should return here.
for (Index index : ((DataStream) indexAbstraction).getFailureIndices().getIndices()) {
indicesAndAliases.add(index.getName());
}
}
}
}

View file

@ -158,6 +158,8 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
final String otherDataStreamName = "logs-foo";
IndexMetadata dataStreamIndex1 = DataStreamTestHelper.createBackingIndex(dataStreamName, 1).build();
IndexMetadata dataStreamIndex2 = DataStreamTestHelper.createBackingIndex(dataStreamName, 2).build();
IndexMetadata dataStreamFailureStore1 = DataStreamTestHelper.createFailureStore(dataStreamName, 1).build();
IndexMetadata dataStreamFailureStore2 = DataStreamTestHelper.createFailureStore(dataStreamName, 2).build();
IndexMetadata dataStreamIndex3 = DataStreamTestHelper.createBackingIndex(otherDataStreamName, 1).build();
Metadata metadata = Metadata.builder()
.put(
@ -217,7 +219,15 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
.put(dataStreamIndex1, true)
.put(dataStreamIndex2, true)
.put(dataStreamIndex3, true)
.put(newInstance(dataStreamName, List.of(dataStreamIndex1.getIndex(), dataStreamIndex2.getIndex())))
.put(dataStreamFailureStore1, true)
.put(dataStreamFailureStore2, true)
.put(
newInstance(
dataStreamName,
List.of(dataStreamIndex1.getIndex(), dataStreamIndex2.getIndex()),
List.of(dataStreamFailureStore1.getIndex(), dataStreamFailureStore2.getIndex())
)
)
.put(newInstance(otherDataStreamName, List.of(dataStreamIndex3.getIndex())))
.put(indexBuilder(securityIndexName).settings(settings))
.build();
@ -337,7 +347,9 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
new RoleDescriptor(
"backing_index_test_wildcards",
null,
new IndicesPrivileges[] { IndicesPrivileges.builder().indices(".ds-logs*").privileges("all").build() },
new IndicesPrivileges[] {
IndicesPrivileges.builder().indices(".ds-logs*").privileges("all").build(),
IndicesPrivileges.builder().indices(".fs-logs*").privileges("all").build() },
null
)
);
@ -347,7 +359,8 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
"backing_index_test_name",
null,
new IndicesPrivileges[] {
IndicesPrivileges.builder().indices(dataStreamIndex1.getIndex().getName()).privileges("all").build() },
IndicesPrivileges.builder().indices(dataStreamIndex1.getIndex().getName()).privileges("all").build(),
IndicesPrivileges.builder().indices(dataStreamFailureStore1.getIndex().getName()).privileges("all").build() },
null
)
);
@ -432,6 +445,38 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
);
}
public void testSelectorsDoNotImpactWildcardDetection() {
ShardSearchRequest request = mock(ShardSearchRequest.class);
when(request.indices()).thenReturn(new String[] { "index*::data" });
IllegalArgumentException exception = expectThrows(
IllegalArgumentException.class,
() -> defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards(TransportSearchAction.TYPE.name() + "[s]", request)
);
assertThat(
exception,
throwableWithMessage(
"the action indices:data/read/search[s] does not support wildcards;"
+ " the provided index expression(s) [index*::data] are not allowed"
)
);
}
public void testWildcardSelectorsAreNotAllowedInShardLevelRequests() {
ShardSearchRequest request = mock(ShardSearchRequest.class);
when(request.indices()).thenReturn(new String[] { "index10::*" });
IllegalArgumentException exception = expectThrows(
IllegalArgumentException.class,
() -> defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards(TransportSearchAction.TYPE.name() + "[s]", request)
);
assertThat(
exception,
throwableWithMessage(
"the action indices:data/read/search[s] does not support wildcard selectors;"
+ " the provided index expression(s) [index10::*] are not allowed"
)
);
}
public void testAllIsNotAllowedInShardLevelRequests() {
ShardSearchRequest request = mock(ShardSearchRequest.class);
final boolean literalAll = randomBoolean();
@ -460,6 +505,23 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
);
}
public void testSelectorsDoNotImpactAllPatternDetection() {
ShardSearchRequest request = mock(ShardSearchRequest.class);
when(request.indices()).thenReturn(new String[] { "_all::data" });
IllegalArgumentException exception = expectThrows(
IllegalArgumentException.class,
() -> defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards(TransportSearchAction.TYPE.name() + "[s]", request)
);
assertThat(
exception,
throwableWithMessage(
"the action indices:data/read/search[s] does not support accessing all indices;"
+ " the provided index expression [_all::data] is not allowed"
)
);
}
public void testResolveIndicesAndAliasesWithoutWildcardsWithSingleIndexNoWildcardsRequest() {
// test 1: matching local index
{
@ -2305,12 +2367,13 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
public void testBackingIndicesAreVisibleWhenIncludedByRequestWithWildcard() {
final User user = new User("data-stream-tester3", "data_stream_test3");
SearchRequest request = new SearchRequest(".ds-logs*");
boolean failureStore = randomBoolean();
SearchRequest request = new SearchRequest(failureStore ? ".fs-logs*" : ".ds-logs*");
assertThat(request, instanceOf(IndicesRequest.Replaceable.class));
assertThat(request.includeDataStreams(), is(true));
// data streams and their backing indices should be included in the authorized list
List<String> expectedDataStreams = List.of("logs-foo", "logs-foobar");
List<String> expectedDataStreams = failureStore ? List.of("logs-foobar") : List.of("logs-foo", "logs-foobar");
final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name(), request);
for (String dsName : expectedDataStreams) {
DataStream dataStream = metadata.dataStreams().get(dsName);
@ -2320,6 +2383,10 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
assertThat(authorizedIndices.all().get(), hasItem(i.getName()));
assertThat(authorizedIndices.check(i.getName()), is(true));
}
for (Index i : dataStream.getFailureIndices().getIndices()) {
assertThat(authorizedIndices.all().get(), hasItem(i.getName()));
assertThat(authorizedIndices.check(i.getName()), is(true));
}
}
// data streams should _not_ be included in the resolved list because they do not match the pattern but their backing indices
@ -2333,7 +2400,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
for (String dsName : expectedDataStreams) {
DataStream dataStream = metadata.dataStreams().get(dsName);
assertThat(resolvedIndices.getLocal(), not(hasItem(dsName)));
for (Index i : dataStream.getIndices()) {
for (Index i : dataStream.getDataStreamIndices(failureStore).getIndices()) {
assertThat(resolvedIndices.getLocal(), hasItem(i.getName()));
}
}
@ -2352,12 +2419,14 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
assertThat(authorizedIndices.all().get(), hasItem(dataStreamName));
assertThat(authorizedIndices.check(dataStreamName), is(true));
DataStream dataStream = metadata.dataStreams().get(dataStreamName);
assertThat(authorizedIndices.all().get(), hasItem(dataStreamName));
assertThat(authorizedIndices.check(dataStreamName), is(true));
for (Index i : dataStream.getIndices()) {
assertThat(authorizedIndices.all().get(), hasItem(i.getName()));
assertThat(authorizedIndices.check(i.getName()), is(true));
}
for (Index i : dataStream.getFailureIndices().getIndices()) {
assertThat(authorizedIndices.all().get(), hasItem(i.getName()));
assertThat(authorizedIndices.check(i.getName()), is(true));
}
// neither data streams nor their backing indices will be in the resolved list since the request does not support data streams
// and the backing indices do not match the requested name
@ -2371,11 +2440,15 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
for (Index i : dataStream.getIndices()) {
assertThat(resolvedIndices.getLocal(), hasItem(i.getName()));
}
for (Index i : dataStream.getFailureIndices().getIndices()) {
assertThat(resolvedIndices.getLocal(), hasItem(i.getName()));
}
}
public void testDataStreamNotAuthorizedWhenBackingIndicesAreAuthorizedViaWildcardAndRequestThatIncludesDataStreams() {
final User user = new User("data-stream-tester2", "backing_index_test_wildcards");
String indexName = ".ds-logs-foobar-*";
boolean failureStore = randomBoolean();
String indexName = failureStore ? ".fs-logs-foobar-*" : ".ds-logs-foobar-*";
SearchRequest request = new SearchRequest(indexName);
assertThat(request, instanceOf(IndicesRequest.Replaceable.class));
assertThat(request.includeDataStreams(), is(true));
@ -2393,6 +2466,10 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
assertThat(authorizedIndices.all().get(), hasItem(i.getName()));
assertThat(authorizedIndices.check(i.getName()), is(true));
}
for (Index i : dataStream.getFailureIndices().getIndices()) {
assertThat(authorizedIndices.all().get(), hasItem(i.getName()));
assertThat(authorizedIndices.check(i.getName()), is(true));
}
// only the backing indices will be in the resolved list since the request does not support data streams
// but the backing indices match the requested pattern
@ -2403,14 +2480,15 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
authorizedIndices
);
assertThat(resolvedIndices.getLocal(), not(hasItem(dataStream.getName())));
for (Index i : dataStream.getIndices()) {
for (Index i : dataStream.getDataStreamIndices(failureStore).getIndices()) {
assertThat(resolvedIndices.getLocal(), hasItem(i.getName()));
}
}
public void testDataStreamNotAuthorizedWhenBackingIndicesAreAuthorizedViaNameAndRequestThatIncludesDataStreams() {
final User user = new User("data-stream-tester2", "backing_index_test_name");
String indexName = ".ds-logs-foobar-*";
boolean failureStore = randomBoolean();
String indexName = failureStore ? ".fs-logs-foobar-*" : ".ds-logs-foobar-*";
SearchRequest request = new SearchRequest(indexName);
assertThat(request, instanceOf(IndicesRequest.Replaceable.class));
assertThat(request.includeDataStreams(), is(true));
@ -2420,8 +2498,12 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, GetAliasesAction.NAME, request);
assertThat(authorizedIndices.all().get(), not(hasItem("logs-foobar")));
assertThat(authorizedIndices.check("logs-foobar"), is(false));
assertThat(authorizedIndices.all().get(), contains(DataStream.getDefaultBackingIndexName("logs-foobar", 1)));
assertThat(authorizedIndices.check(DataStream.getDefaultBackingIndexName("logs-foobar", 1)), is(true));
String expectedIndex = failureStore
? DataStream.getDefaultFailureStoreName("logs-foobar", 1, System.currentTimeMillis())
: DataStream.getDefaultBackingIndexName("logs-foobar", 1);
assertThat(authorizedIndices.all().get(), hasItem(expectedIndex));
assertThat(authorizedIndices.check(expectedIndex), is(true));
// only the single backing index will be in the resolved list since the request does not support data streams
// but one of the backing indices matched the requested pattern
@ -2432,12 +2514,13 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
authorizedIndices
);
assertThat(resolvedIndices.getLocal(), not(hasItem("logs-foobar")));
assertThat(resolvedIndices.getLocal(), contains(DataStream.getDefaultBackingIndexName("logs-foobar", 1)));
assertThat(resolvedIndices.getLocal(), contains(expectedIndex));
}
public void testDataStreamNotAuthorizedWhenBackingIndicesAreAuthorizedViaWildcardAndRequestThatExcludesDataStreams() {
final User user = new User("data-stream-tester2", "backing_index_test_wildcards");
String indexName = ".ds-logs-foobar-*";
boolean failureStore = randomBoolean();
String indexName = failureStore ? ".fs-logs-foobar-*" : ".ds-logs-foobar-*";
GetAliasesRequest request = new GetAliasesRequest(indexName);
assertThat(request, instanceOf(IndicesRequest.Replaceable.class));
assertThat(request.includeDataStreams(), is(true));
@ -2455,6 +2538,10 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
assertThat(authorizedIndices.all().get(), hasItem(i.getName()));
assertThat(authorizedIndices.check(i.getName()), is(true));
}
for (Index i : dataStream.getFailureIndices().getIndices()) {
assertThat(authorizedIndices.all().get(), hasItem(i.getName()));
assertThat(authorizedIndices.check(i.getName()), is(true));
}
// only the backing indices will be in the resolved list since the request does not support data streams
// but the backing indices match the requested pattern
@ -2465,25 +2552,29 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
authorizedIndices
);
assertThat(resolvedIndices.getLocal(), not(hasItem(dataStream.getName())));
for (Index i : dataStream.getIndices()) {
for (Index i : dataStream.getDataStreamIndices(failureStore).getIndices()) {
assertThat(resolvedIndices.getLocal(), hasItem(i.getName()));
}
}
public void testDataStreamNotAuthorizedWhenBackingIndicesAreAuthorizedViaNameAndRequestThatExcludesDataStreams() {
final User user = new User("data-stream-tester2", "backing_index_test_name");
String indexName = ".ds-logs-foobar-*";
boolean failureStore = randomBoolean();
String indexName = failureStore ? ".fs-logs-foobar-*" : ".ds-logs-foobar-*";
GetAliasesRequest request = new GetAliasesRequest(indexName);
assertThat(request, instanceOf(IndicesRequest.Replaceable.class));
assertThat(request.includeDataStreams(), is(true));
// data streams should _not_ be in the authorized list but a single backing index that matched the requested pattern
// and the authorized name should be in the list
String expectedIndex = failureStore
? DataStream.getDefaultFailureStoreName("logs-foobar", 1, System.currentTimeMillis())
: DataStream.getDefaultBackingIndexName("logs-foobar", 1);
final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, GetAliasesAction.NAME, request);
assertThat(authorizedIndices.all().get(), not(hasItem("logs-foobar")));
assertThat(authorizedIndices.check("logs-foobar"), is(false));
assertThat(authorizedIndices.all().get(), contains(DataStream.getDefaultBackingIndexName("logs-foobar", 1)));
assertThat(authorizedIndices.check(DataStream.getDefaultBackingIndexName("logs-foobar", 1)), is(true));
assertThat(authorizedIndices.all().get(), hasItem(expectedIndex));
assertThat(authorizedIndices.check(expectedIndex), is(true));
// only the single backing index will be in the resolved list since the request does not support data streams
// but one of the backing indices matched the requested pattern
@ -2494,7 +2585,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase {
authorizedIndices
);
assertThat(resolvedIndices.getLocal(), not(hasItem("logs-foobar")));
assertThat(resolvedIndices.getLocal(), contains(DataStream.getDefaultBackingIndexName("logs-foobar", 1)));
assertThat(resolvedIndices.getLocal(), hasItem(expectedIndex));
}
public void testResolveSearchShardRequestAgainstDataStream() {

View file

@ -145,6 +145,7 @@ enrich documents over _bulk via an alias:
- '{"baz": "lazy", "c": 2}'
- '{"index": {"_id": "3"}}'
- '{"baz": "slow", "c": 3}'
- match: { errors: false }
- do:
get: