mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-04-24 23:27:25 -04:00
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:
parent
82b1f2a205
commit
c3839e1f76
73 changed files with 2377 additions and 1073 deletions
|
@ -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"/]
|
||||
|
|
|
@ -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.")
|
||||
})
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -50,7 +50,7 @@ public class DeleteDataStreamLifecycleAction {
|
|||
.allowAliasToMultipleIndices(false)
|
||||
.allowClosedIndices(true)
|
||||
.ignoreThrottled(false)
|
||||
.allowFailureIndices(false)
|
||||
.allowSelectors(false)
|
||||
.build()
|
||||
)
|
||||
.build();
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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()
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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/" }
|
||||
|
|
|
@ -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":{
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -74,7 +74,7 @@ public class IndicesAliasesRequest extends AcknowledgedRequest<IndicesAliasesReq
|
|||
.allowAliasToMultipleIndices(true)
|
||||
.allowClosedIndices(true)
|
||||
.ignoreThrottled(false)
|
||||
.allowFailureIndices(true)
|
||||
.allowSelectors(false)
|
||||
.build()
|
||||
)
|
||||
.build();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -43,7 +43,7 @@ public class DeleteIndexRequest extends AcknowledgedRequest<DeleteIndexRequest>
|
|||
.allowAliasToMultipleIndices(false)
|
||||
.allowClosedIndices(true)
|
||||
.ignoreThrottled(false)
|
||||
.allowFailureIndices(true)
|
||||
.allowSelectors(false)
|
||||
.build()
|
||||
)
|
||||
.build();
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -82,7 +82,7 @@ public class PutMappingRequest extends AcknowledgedRequest<PutMappingRequest> im
|
|||
.allowClosedIndices(true)
|
||||
.allowAliasToMultipleIndices(true)
|
||||
.ignoreThrottled(false)
|
||||
.allowFailureIndices(false)
|
||||
.allowSelectors(false)
|
||||
)
|
||||
.build();
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
|
|
|
@ -61,7 +61,7 @@ public class DeleteDataStreamAction extends ActionType<AcknowledgedResponse> {
|
|||
.allowAliasToMultipleIndices(false)
|
||||
.allowClosedIndices(true)
|
||||
.ignoreThrottled(false)
|
||||
.allowFailureIndices(true)
|
||||
.allowSelectors(false)
|
||||
.build()
|
||||
)
|
||||
.build();
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -63,7 +63,7 @@ public class GetDataStreamLifecycleAction {
|
|||
.allowAliasToMultipleIndices(false)
|
||||
.allowClosedIndices(true)
|
||||
.ignoreThrottled(false)
|
||||
.allowFailureIndices(true)
|
||||
.allowSelectors(false)
|
||||
.build()
|
||||
)
|
||||
.build();
|
||||
|
|
|
@ -94,7 +94,7 @@ public class PutDataStreamLifecycleAction {
|
|||
.allowAliasToMultipleIndices(false)
|
||||
.allowClosedIndices(true)
|
||||
.ignoreThrottled(false)
|
||||
.allowFailureIndices(false)
|
||||
.allowSelectors(false)
|
||||
.build()
|
||||
)
|
||||
.build();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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() : "")
|
||||
+ ']';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()) {
|
||||
|
|
File diff suppressed because it is too large
Load diff
|
@ -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()
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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 -> {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
||||
|
|
|
@ -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]]]"
|
||||
)
|
||||
)
|
||||
);
|
||||
|
|
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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:
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue