diff --git a/server/src/main/java/org/elasticsearch/action/ActionModule.java b/server/src/main/java/org/elasticsearch/action/ActionModule.java index e991d3f443e57..27d441830c5c0 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/server/src/main/java/org/elasticsearch/action/ActionModule.java @@ -666,7 +666,7 @@ public void initRestHandlers(Supplier nodesInCluster) { registerHandler.accept(new RestMasterAction(settings, restController)); registerHandler.accept(new RestNodesAction(settings, restController)); registerHandler.accept(new RestTasksAction(settings, restController, nodesInCluster)); - registerHandler.accept(new RestIndicesAction(settings, restController, indexNameExpressionResolver)); + registerHandler.accept(new RestIndicesAction(settings, restController)); registerHandler.accept(new RestSegmentsAction(settings, restController)); // Fully qualified to prevent interference with rest.action.count.RestCountAction registerHandler.accept(new org.elasticsearch.rest.action.cat.RestCountAction(settings, restController)); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java index 788de8e715480..32fb62b82033f 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java @@ -19,54 +19,58 @@ package org.elasticsearch.rest.action.cat; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; +import org.elasticsearch.action.admin.indices.settings.get.GetSettingsRequest; +import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse; import org.elasticsearch.action.admin.indices.stats.CommonStats; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequest; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.support.GroupedActionListener; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.client.Requests; import org.elasticsearch.client.node.NodeClient; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterIndexHealth; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.Strings; import org.elasticsearch.common.Table; -import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.time.DateFormatter; -import org.elasticsearch.index.Index; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; -import org.elasticsearch.rest.action.RestActionListener; import org.elasticsearch.rest.action.RestResponseListener; import java.time.Instant; import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Locale; +import java.util.Map; import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import static org.elasticsearch.action.support.master.MasterNodeRequest.DEFAULT_MASTER_NODE_TIMEOUT; import static org.elasticsearch.rest.RestRequest.Method.GET; public class RestIndicesAction extends AbstractCatAction { private static final DateFormatter STRICT_DATE_TIME_FORMATTER = DateFormatter.forPattern("strict_date_time"); - private final IndexNameExpressionResolver indexNameExpressionResolver; - public RestIndicesAction(Settings settings, RestController controller, IndexNameExpressionResolver indexNameExpressionResolver) { + public RestIndicesAction(Settings settings, RestController controller) { super(settings); - this.indexNameExpressionResolver = indexNameExpressionResolver; controller.registerHandler(GET, "/_cat/indices", this); controller.registerHandler(GET, "/_cat/indices/{index}", this); } @@ -85,50 +89,147 @@ protected void documentation(StringBuilder sb) { @Override public RestChannelConsumer doCatRequest(final RestRequest request, final NodeClient client) { final String[] indices = Strings.splitStringByCommaToArray(request.param("index")); - final ClusterStateRequest clusterStateRequest = new ClusterStateRequest(); - clusterStateRequest.clear().indices(indices).metaData(true); - clusterStateRequest.local(request.paramAsBoolean("local", clusterStateRequest.local())); - clusterStateRequest.masterNodeTimeout(request.paramAsTime("master_timeout", clusterStateRequest.masterNodeTimeout())); - final IndicesOptions strictExpandIndicesOptions = IndicesOptions.strictExpand(); - clusterStateRequest.indicesOptions(strictExpandIndicesOptions); - - return channel -> client.admin().cluster().state(clusterStateRequest, new RestActionListener(channel) { + final IndicesOptions indicesOptions = IndicesOptions.strictExpand(); + final boolean local = request.paramAsBoolean("local", false); + final TimeValue masterNodeTimeout = request.paramAsTime("master_timeout", DEFAULT_MASTER_NODE_TIMEOUT); + final boolean includeUnloadedSegments = request.paramAsBoolean("include_unloaded_segments", false); + + return channel -> { + final ActionListener listener = ActionListener.notifyOnce(new RestResponseListener
(channel) { + @Override + public RestResponse buildResponse(final Table table) throws Exception { + return RestTable.buildResponse(table, channel); + } + }); + + sendGetSettingsRequest(indices, indicesOptions, local, masterNodeTimeout, client, new ActionListener() { + @Override + public void onResponse(final GetSettingsResponse getSettingsResponse) { + final GroupedActionListener groupedListener = createGroupedListener(request, 4, listener); + groupedListener.onResponse(getSettingsResponse); + + // Indices that were successfully resolved during the get settings request might be deleted when the subsequent cluster + // state, cluster health and indices stats requests execute. We have to distinguish two cases: + // 1) the deleted index was explicitly passed as parameter to the /_cat/indices request. In this case we want the + // subsequent requests to fail. + // 2) the deleted index was resolved as part of a wildcard or _all. In this case, we want the subsequent requests not to + // fail on the deleted index (as we want to ignore wildcards that cannot be resolved). + // This behavior can be ensured by letting the cluster state, cluster health and indices stats requests re-resolve the + // index names with the same indices options that we used for the initial cluster state request (strictExpand). + sendIndicesStatsRequest(indices, indicesOptions, includeUnloadedSegments, client, + ActionListener.wrap(groupedListener::onResponse, groupedListener::onFailure)); + sendClusterStateRequest(indices, indicesOptions, local, masterNodeTimeout, client, + ActionListener.wrap(groupedListener::onResponse, groupedListener::onFailure)); + sendClusterHealthRequest(indices, indicesOptions, local, masterNodeTimeout, client, + ActionListener.wrap(groupedListener::onResponse, groupedListener::onFailure)); + } + + @Override + public void onFailure(final Exception e) { + listener.onFailure(e); + } + }); + }; + } + + /** + * We're using the Get Settings API here to resolve the authorized indices for the user. + * This is because the Cluster State and Cluster Health APIs do not filter output based + * on index privileges, so they can't be used to determine which indices are authorized + * or not. On top of this, the Indices Stats API cannot be used either to resolve indices + * as it does not provide information for all existing indices (for example recovering + * indices or non replicated closed indices are not reported in indices stats response). + */ + private void sendGetSettingsRequest(final String[] indices, + final IndicesOptions indicesOptions, + final boolean local, + final TimeValue masterNodeTimeout, + final NodeClient client, + final ActionListener listener) { + final GetSettingsRequest request = new GetSettingsRequest(); + request.indices(indices); + request.indicesOptions(indicesOptions); + request.local(local); + request.masterNodeTimeout(masterNodeTimeout); + request.names(IndexSettings.INDEX_SEARCH_THROTTLED.getKey()); + + client.admin().indices().getSettings(request, listener); + } + + private void sendClusterStateRequest(final String[] indices, + final IndicesOptions indicesOptions, + final boolean local, + final TimeValue masterNodeTimeout, + final NodeClient client, + final ActionListener listener) { + + final ClusterStateRequest request = new ClusterStateRequest(); + request.indices(indices); + request.indicesOptions(indicesOptions); + request.local(local); + request.masterNodeTimeout(masterNodeTimeout); + + client.admin().cluster().state(request, listener); + } + + private void sendClusterHealthRequest(final String[] indices, + final IndicesOptions indicesOptions, + final boolean local, + final TimeValue masterNodeTimeout, + final NodeClient client, + final ActionListener listener) { + + final ClusterHealthRequest request = new ClusterHealthRequest(); + request.indices(indices); + request.indicesOptions(indicesOptions); + request.local(local); + request.masterNodeTimeout(masterNodeTimeout); + + client.admin().cluster().health(request, listener); + } + + private void sendIndicesStatsRequest(final String[] indices, + final IndicesOptions indicesOptions, + final boolean includeUnloadedSegments, + final NodeClient client, + final ActionListener listener) { + + final IndicesStatsRequest request = new IndicesStatsRequest(); + request.indices(indices); + request.indicesOptions(indicesOptions); + request.all(); + request.includeUnloadedSegments(includeUnloadedSegments); + + client.admin().indices().stats(request, listener); + } + + private GroupedActionListener createGroupedListener(final RestRequest request, final int size, + final ActionListener
listener) { + return new GroupedActionListener<>(new ActionListener>() { @Override - public void processResponse(final ClusterStateResponse clusterStateResponse) { - final ClusterState clusterState = clusterStateResponse.getState(); - final IndexMetaData[] indicesMetaData = getOrderedIndexMetaData(indices, clusterState, strictExpandIndicesOptions); - // Indices that were successfully resolved during the cluster state request might be deleted when the subsequent cluster - // health and indices stats requests execute. We have to distinguish two cases: - // 1) the deleted index was explicitly passed as parameter to the /_cat/indices request. In this case we want the subsequent - // requests to fail. - // 2) the deleted index was resolved as part of a wildcard or _all. In this case, we want the subsequent requests not to - // fail on the deleted index (as we want to ignore wildcards that cannot be resolved). - // This behavior can be ensured by letting the cluster health and indices stats requests re-resolve the index names with the - // same indices options that we used for the initial cluster state request (strictExpand). - final ClusterHealthRequest clusterHealthRequest = Requests.clusterHealthRequest(indices); - clusterHealthRequest.indicesOptions(strictExpandIndicesOptions); - clusterHealthRequest.local(request.paramAsBoolean("local", clusterHealthRequest.local())); - - client.admin().cluster().health(clusterHealthRequest, new RestActionListener(channel) { - @Override - public void processResponse(final ClusterHealthResponse clusterHealthResponse) { - final IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest(); - indicesStatsRequest.indices(indices); - indicesStatsRequest.indicesOptions(strictExpandIndicesOptions); - indicesStatsRequest.all(); - indicesStatsRequest.includeUnloadedSegments(request.paramAsBoolean("include_unloaded_segments", false)); - - client.admin().indices().stats(indicesStatsRequest, new RestResponseListener(channel) { - @Override - public RestResponse buildResponse(IndicesStatsResponse indicesStatsResponse) throws Exception { - final Table tab = buildTable(request, indicesMetaData, clusterHealthResponse, indicesStatsResponse); - return RestTable.buildResponse(tab, channel); - } - }); - } - }); + public void onResponse(final Collection responses) { + GetSettingsResponse settingsResponse = extractResponse(responses, GetSettingsResponse.class); + Map indicesSettings = StreamSupport.stream(settingsResponse.getIndexToSettings().spliterator(), false) + .collect(Collectors.toMap(cursor -> cursor.key, cursor -> cursor.value)); + + ClusterStateResponse stateResponse = extractResponse(responses, ClusterStateResponse.class); + Map indicesStates = StreamSupport.stream(stateResponse.getState().getMetaData().spliterator(), false) + .collect(Collectors.toMap(indexMetaData -> indexMetaData.getIndex().getName(), Function.identity())); + + ClusterHealthResponse healthResponse = extractResponse(responses, ClusterHealthResponse.class); + Map indicesHealths = healthResponse.getIndices(); + + IndicesStatsResponse statsResponse = extractResponse(responses, IndicesStatsResponse.class); + Map indicesStats = statsResponse.getIndices(); + + listener.onResponse(buildTable(request, indicesSettings, indicesHealths, indicesStats, indicesStates)); } - }); + + @Override + public void onFailure(final Exception e) { + listener.onFailure(e); + } + }, size); } private static final Set RESPONSE_PARAMS; @@ -393,18 +494,35 @@ protected Table getTableWithHeader(final RestRequest request) { // package private for testing Table buildTable(final RestRequest request, - final IndexMetaData[] indicesMetaData, - final ClusterHealthResponse clusterHealthResponse, - final IndicesStatsResponse indicesStatsResponse) { - final String healthParam = request.param("health"); + final Map indicesSettings, + final Map indicesHealths, + final Map indicesStats, + final Map indicesMetaDatas) { + final String healthParam = request.param("health"); final Table table = getTableWithHeader(request); - for (IndexMetaData indexMetaData : indicesMetaData) { - final String indexName = indexMetaData.getIndex().getName(); - final ClusterIndexHealth indexHealth = clusterHealthResponse.getIndices().get(indexName); - final IndexStats indexStats = indicesStatsResponse.getIndices().get(indexName); + + indicesSettings.forEach((indexName, settings) -> { + if (indicesMetaDatas.containsKey(indexName) == false) { + // the index exists in the Get Indices response but is not present in the cluster state: + // it is likely that the index was deleted in the meanwhile, so we ignore it. + return; + } + + final IndexMetaData indexMetaData = indicesMetaDatas.get(indexName); final IndexMetaData.State indexState = indexMetaData.getState(); - final boolean searchThrottled = IndexSettings.INDEX_SEARCH_THROTTLED.get(indexMetaData.getSettings()); + final IndexStats indexStats = indicesStats.get(indexName); + final boolean searchThrottled = IndexSettings.INDEX_SEARCH_THROTTLED.get(settings); + + final String health; + final ClusterIndexHealth indexHealth = indicesHealths.get(indexName); + if (indexHealth != null) { + health = indexHealth.getStatus().toString().toLowerCase(Locale.ROOT); + } else if (indexStats != null) { + health = "red*"; + } else { + health = ""; + } if (healthParam != null) { final ClusterHealthStatus healthStatusFilter = ClusterHealthStatus.fromString(healthParam); @@ -413,44 +531,26 @@ Table buildTable(final RestRequest request, // index health is known but does not match the one requested skip = indexHealth.getStatus() != healthStatusFilter; } else { - // index health is unknown, skip if we don't explicitly request RED health or if the index is closed but not replicated - skip = ClusterHealthStatus.RED != healthStatusFilter || indexState == IndexMetaData.State.CLOSE; + // index health is unknown, skip if we don't explicitly request RED health + skip = ClusterHealthStatus.RED != healthStatusFilter; } if (skip) { - continue; + return; } } - // the open index is present in the cluster state but is not returned in the indices stats API - if (indexStats == null && indexState != IndexMetaData.State.CLOSE) { - // the index stats API is called last, after cluster state and cluster health. If the index stats - // has not resolved the same open indices as the initial cluster state call, then the indices might - // have been removed in the meantime or, more likely, are unauthorized. This is because the cluster - // state exposes everything, even unauthorized indices, which are not exposed in APIs. - // We ignore such an index instead of displaying it with an empty stats. - continue; - } - final CommonStats primaryStats; final CommonStats totalStats; - if (indexState == IndexMetaData.State.CLOSE) { - // empty stats for closed indices, but their names are displayed + if (indexStats == null || indexState == IndexMetaData.State.CLOSE) { + // TODO: expose docs stats for replicated closed indices primaryStats = new CommonStats(); totalStats = new CommonStats(); } else { primaryStats = indexStats.getPrimaries(); totalStats = indexStats.getTotal(); } - table.startRow(); - - String health = null; - if (indexHealth != null) { - health = indexHealth.getStatus().toString().toLowerCase(Locale.ROOT); - } else if (indexStats != null) { - health = "red*"; - } table.addCell(health); table.addCell(indexState.toString().toLowerCase(Locale.ROOT)); table.addCell(indexName); @@ -648,25 +748,13 @@ Table buildTable(final RestRequest request, table.addCell(searchThrottled); table.endRow(); - } + }); return table; } - // package private for testing - IndexMetaData[] getOrderedIndexMetaData(String[] indicesExpression, ClusterState clusterState, IndicesOptions indicesOptions) { - final Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(clusterState, indicesOptions, indicesExpression); - // concreteIndices should contain exactly the indices in state.metaData() that were selected by clusterStateRequest using the - // same indices option (IndicesOptions.strictExpand()). We select the indices again here so that they can be displayed in the - // resulting table in the requesting order. - assert concreteIndices.length == clusterState.metaData().getIndices().size(); - final ImmutableOpenMap indexMetaDataMap = clusterState.metaData().getIndices(); - final IndexMetaData[] indicesMetaData = new IndexMetaData[concreteIndices.length]; - // select the index metadata in the requested order, so that the response can display the indices in the resulting table - // in the requesting order. - for (int i = 0; i < concreteIndices.length; i++) { - indicesMetaData[i] = indexMetaDataMap.get(concreteIndices[i].getName()); - } - return indicesMetaData; + @SuppressWarnings("unchecked") + private static A extractResponse(final Collection responses, Class c) { + return (A) responses.stream().filter(c::isInstance).findFirst().get(); } } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverActionTests.java index ba6622ec2d142..c24e0c65e00ce 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/rollover/TransportRolloverActionTests.java @@ -27,6 +27,8 @@ import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequestBuilder; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; +import org.elasticsearch.action.admin.indices.stats.IndicesStatsTests; +import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.PlainActionFuture; @@ -43,6 +45,9 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService; import org.elasticsearch.cluster.metadata.MetaDataIndexAliasesService; +import org.elasticsearch.cluster.routing.RecoverySource; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; @@ -50,13 +55,29 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.index.cache.query.QueryCacheStats; +import org.elasticsearch.index.cache.request.RequestCacheStats; +import org.elasticsearch.index.engine.SegmentsStats; +import org.elasticsearch.index.fielddata.FieldDataStats; +import org.elasticsearch.index.flush.FlushStats; +import org.elasticsearch.index.get.GetStats; +import org.elasticsearch.index.merge.MergeStats; +import org.elasticsearch.index.refresh.RefreshStats; +import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.shard.DocsStats; -import org.elasticsearch.rest.action.cat.RestIndicesActionTests; +import org.elasticsearch.index.shard.IndexingStats; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.ShardPath; +import org.elasticsearch.index.store.StoreStats; +import org.elasticsearch.index.warmer.WarmerStats; +import org.elasticsearch.search.suggest.completion.CompletionStats; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.mockito.ArgumentCaptor; +import java.nio.file.Path; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -64,6 +85,7 @@ import java.util.Map; import java.util.Set; +import static java.util.Collections.emptyList; import static org.elasticsearch.action.admin.indices.rollover.TransportRolloverAction.evaluateConditions; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -189,7 +211,7 @@ public void testEvaluateWithoutMetaData() { .creationDate(System.currentTimeMillis() - TimeValue.timeValueHours(randomIntBetween(5, 10)).getMillis()) .settings(settings) .build(); - IndicesStatsResponse indicesStats = RestIndicesActionTests.randomIndicesStatsResponse(new IndexMetaData[]{metaData}); + IndicesStatsResponse indicesStats = randomIndicesStatsResponse(new IndexMetaData[]{metaData}); Map results2 = evaluateConditions(conditions, null, indicesStats); assertThat(results2.size(), equalTo(3)); results2.forEach((k, v) -> assertFalse(v)); @@ -490,4 +512,42 @@ private static Condition createTestCondition() { when(condition.evaluate(any())).thenReturn(new Condition.Result(condition, true)); return condition; } + + public static IndicesStatsResponse randomIndicesStatsResponse(final IndexMetaData[] indices) { + List shardStats = new ArrayList<>(); + for (final IndexMetaData index : indices) { + int numShards = randomIntBetween(1, 3); + int primaryIdx = randomIntBetween(-1, numShards - 1); // -1 means there is no primary shard. + for (int i = 0; i < numShards; i++) { + ShardId shardId = new ShardId(index.getIndex(), i); + boolean primary = (i == primaryIdx); + Path path = createTempDir().resolve("indices").resolve(index.getIndexUUID()).resolve(String.valueOf(i)); + ShardRouting shardRouting = ShardRouting.newUnassigned(shardId, primary, + primary ? RecoverySource.EmptyStoreRecoverySource.INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null) + ); + shardRouting = shardRouting.initialize("node-0", null, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE); + shardRouting = shardRouting.moveToStarted(); + CommonStats stats = new CommonStats(); + stats.fieldData = new FieldDataStats(); + stats.queryCache = new QueryCacheStats(); + stats.docs = new DocsStats(); + stats.store = new StoreStats(); + stats.indexing = new IndexingStats(); + stats.search = new SearchStats(); + stats.segments = new SegmentsStats(); + stats.merge = new MergeStats(); + stats.refresh = new RefreshStats(); + stats.completion = new CompletionStats(); + stats.requestCache = new RequestCacheStats(); + stats.get = new GetStats(); + stats.flush = new FlushStats(); + stats.warmer = new WarmerStats(); + shardStats.add(new ShardStats(shardRouting, new ShardPath(false, path, path, shardId), stats, null, null, null)); + } + } + return IndicesStatsTests.newIndicesStatsResponse( + shardStats.toArray(new ShardStats[shardStats.size()]), shardStats.size(), shardStats.size(), 0, emptyList() + ); + } } diff --git a/server/src/test/java/org/elasticsearch/rest/action/cat/RestIndicesActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/cat/RestIndicesActionTests.java index b9a42b5cc1c7b..37dc84c126e2e 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/cat/RestIndicesActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/cat/RestIndicesActionTests.java @@ -20,182 +20,154 @@ package org.elasticsearch.rest.action.cat; import org.elasticsearch.Version; -import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.indices.stats.CommonStats; -import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; -import org.elasticsearch.action.admin.indices.stats.IndicesStatsTests; -import org.elasticsearch.action.admin.indices.stats.ShardStats; -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.action.admin.indices.stats.IndexStats; +import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.cluster.health.ClusterIndexHealth; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.routing.RecoverySource; -import org.elasticsearch.cluster.routing.RecoverySource.PeerRecoverySource; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.UnassignedInfo; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.Table; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.index.cache.query.QueryCacheStats; -import org.elasticsearch.index.cache.request.RequestCacheStats; -import org.elasticsearch.index.engine.SegmentsStats; -import org.elasticsearch.index.fielddata.FieldDataStats; -import org.elasticsearch.index.flush.FlushStats; -import org.elasticsearch.index.get.GetStats; -import org.elasticsearch.index.merge.MergeStats; -import org.elasticsearch.index.refresh.RefreshStats; -import org.elasticsearch.index.search.stats.SearchStats; -import org.elasticsearch.index.shard.DocsStats; -import org.elasticsearch.index.shard.IndexingStats; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.shard.ShardPath; -import org.elasticsearch.index.store.StoreStats; -import org.elasticsearch.index.warmer.WarmerStats; import org.elasticsearch.rest.RestController; -import org.elasticsearch.search.suggest.completion.CompletionStats; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.usage.UsageService; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.IntStream; -import static java.util.Collections.emptyList; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; -/** - * Tests for {@link RestIndicesAction} - */ public class RestIndicesActionTests extends ESTestCase { - private IndexMetaData[] buildRandomIndicesMetaData(int numIndices) { - // build a (semi-)random table - final IndexMetaData[] indicesMetaData = new IndexMetaData[numIndices]; + public void testBuildTable() { + final int numIndices = randomIntBetween(3, 20); + final Map indicesSettings = new LinkedHashMap<>(); + final Map indicesMetaDatas = new LinkedHashMap<>(); + final Map indicesHealths = new LinkedHashMap<>(); + final Map indicesStats = new LinkedHashMap<>(); + for (int i = 0; i < numIndices; i++) { - indicesMetaData[i] = IndexMetaData.builder(randomAlphaOfLength(5) + i) - .settings(Settings.builder() - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID())) - .creationDate(System.currentTimeMillis()) - .numberOfShards(1) - .numberOfReplicas(1) - .state(IndexMetaData.State.OPEN) - .build(); - } - return indicesMetaData; - } + String indexName = "index-" + i; - private ClusterState buildClusterState(IndexMetaData[] indicesMetaData) { - final MetaData.Builder metaDataBuilder = MetaData.builder(); - for (IndexMetaData indexMetaData : indicesMetaData) { - metaDataBuilder.put(indexMetaData, false); - } - final MetaData metaData = metaDataBuilder.build(); - final ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) - .metaData(metaData) - .build(); - return clusterState; - } + Settings indexSettings = Settings.builder() + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) + .put(IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), randomBoolean()) + .build(); + indicesSettings.put(indexName, indexSettings); - private ClusterHealthResponse buildClusterHealthResponse(ClusterState clusterState, IndexMetaData[] indicesMetaData) { - final String[] indicesStr = new String[indicesMetaData.length]; - for (int i = 0; i < indicesMetaData.length; i++) { - indicesStr[i] = indicesMetaData[i].getIndex().getName(); - } - final ClusterHealthResponse clusterHealthResponse = new ClusterHealthResponse( - clusterState.getClusterName().value(), indicesStr, clusterState, 0, 0, 0, TimeValue.timeValueMillis(1000L) - ); - return clusterHealthResponse; - } + IndexMetaData.State indexState = randomBoolean() ? IndexMetaData.State.OPEN : IndexMetaData.State.CLOSE; + if (frequently()) { + ClusterHealthStatus healthStatus = randomFrom(ClusterHealthStatus.values()); + int numberOfShards = randomIntBetween(1, 3); + int numberOfReplicas = healthStatus == ClusterHealthStatus.YELLOW ? 1 : randomInt(1); + IndexMetaData indexMetaData = IndexMetaData.builder(indexName) + .settings(indexSettings) + .creationDate(System.currentTimeMillis()) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + .state(indexState) + .build(); + indicesMetaDatas.put(indexName, indexMetaData); - public void testBuildTable() { - final Settings settings = Settings.EMPTY; - UsageService usageService = new UsageService(); - final RestController restController = new RestController(Collections.emptySet(), null, null, null, usageService); - final RestIndicesAction action = new RestIndicesAction(settings, restController, new IndexNameExpressionResolver()); + if (frequently()) { + Index index = indexMetaData.getIndex(); + IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index); + switch (randomFrom(ClusterHealthStatus.values())) { + case GREEN: + IntStream.range(0, numberOfShards) + .mapToObj(n -> new ShardId(index, n)) + .map(shardId -> TestShardRouting.newShardRouting(shardId, "nodeA", true, ShardRoutingState.STARTED)) + .forEach(indexRoutingTable::addShard); + if (numberOfReplicas > 0) { + IntStream.range(0, numberOfShards) + .mapToObj(n -> new ShardId(index, n)) + .map(shardId -> TestShardRouting.newShardRouting(shardId, "nodeB", false, ShardRoutingState.STARTED)) + .forEach(indexRoutingTable::addShard); + } + break; + case YELLOW: + IntStream.range(0, numberOfShards) + .mapToObj(n -> new ShardId(index, n)) + .map(shardId -> TestShardRouting.newShardRouting(shardId, "nodeA", true, ShardRoutingState.STARTED)) + .forEach(indexRoutingTable::addShard); + if (numberOfReplicas > 0) { + IntStream.range(0, numberOfShards) + .mapToObj(n -> new ShardId(index, n)) + .map(shardId -> TestShardRouting.newShardRouting(shardId, null, false, ShardRoutingState.UNASSIGNED)) + .forEach(indexRoutingTable::addShard); + } + break; + case RED: + break; + } + indicesHealths.put(indexName, new ClusterIndexHealth(indexMetaData, indexRoutingTable.build())); - final IndexMetaData[] generatedIndicesMetaData = buildRandomIndicesMetaData(randomIntBetween(1, 5)); - final ClusterState clusterState = buildClusterState(generatedIndicesMetaData); - final ClusterHealthResponse clusterHealthResponse = buildClusterHealthResponse(clusterState, generatedIndicesMetaData); + if (frequently()) { + IndexStats indexStats = mock(IndexStats.class); + when(indexStats.getPrimaries()).thenReturn(new CommonStats()); + when(indexStats.getTotal()).thenReturn(new CommonStats()); + indicesStats.put(indexName, indexStats); + } + } + } + } - final IndexMetaData[] sortedIndicesMetaData = action.getOrderedIndexMetaData(new String[0], clusterState, - IndicesOptions.strictExpand()); - final IndexMetaData[] smallerSortedIndicesMetaData = removeRandomElement(sortedIndicesMetaData); - final Table table = action.buildTable(new FakeRestRequest(), sortedIndicesMetaData, clusterHealthResponse, - randomIndicesStatsResponse(smallerSortedIndicesMetaData)); + final RestController restController = new RestController(Collections.emptySet(), null, null, null, new UsageService()); + final RestIndicesAction action = new RestIndicesAction(Settings.EMPTY, restController); + final Table table = action.buildTable(new FakeRestRequest(), indicesSettings, indicesHealths, indicesStats, indicesMetaDatas); // now, verify the table is correct - int count = 0; List headers = table.getHeaders(); - assertThat(headers.get(count++).value, equalTo("health")); - assertThat(headers.get(count++).value, equalTo("status")); - assertThat(headers.get(count++).value, equalTo("index")); - assertThat(headers.get(count++).value, equalTo("uuid")); - - List> rows = table.getRows(); - assertThat(rows.size(), equalTo(smallerSortedIndicesMetaData.length)); - // TODO: more to verify (e.g. randomize cluster health, num primaries, num replicas, etc) - for (int i = 0; i < rows.size(); i++) { - count = 0; - final List row = rows.get(i); - assertThat(row.get(count++).value, equalTo("red*")); // all are red because cluster state doesn't have routing entries - assertThat(row.get(count++).value, equalTo("open")); // all are OPEN for now - assertThat(row.get(count++).value, equalTo(smallerSortedIndicesMetaData[i].getIndex().getName())); - assertThat(row.get(count++).value, equalTo(smallerSortedIndicesMetaData[i].getIndexUUID())); - } - } + assertThat(headers.get(0).value, equalTo("health")); + assertThat(headers.get(1).value, equalTo("status")); + assertThat(headers.get(2).value, equalTo("index")); + assertThat(headers.get(3).value, equalTo("uuid")); + assertThat(headers.get(4).value, equalTo("pri")); + assertThat(headers.get(5).value, equalTo("rep")); + + final List> rows = table.getRows(); + assertThat(rows.size(), equalTo(indicesMetaDatas.size())); + + for (final List row : rows) { + final String indexName = (String) row.get(2).value; - public static IndicesStatsResponse randomIndicesStatsResponse(final IndexMetaData[] indices) { - List shardStats = new ArrayList<>(); - for (final IndexMetaData index : indices) { - int numShards = randomIntBetween(1, 3); - int primaryIdx = randomIntBetween(-1, numShards - 1); // -1 means there is no primary shard. - for (int i = 0; i < numShards; i++) { - ShardId shardId = new ShardId(index.getIndex(), i); - boolean primary = (i == primaryIdx); - Path path = createTempDir().resolve("indices").resolve(index.getIndexUUID()).resolve(String.valueOf(i)); - ShardRouting shardRouting = ShardRouting.newUnassigned(shardId, primary, - primary ? RecoverySource.EmptyStoreRecoverySource.INSTANCE : PeerRecoverySource.INSTANCE, - new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null) - ); - shardRouting = shardRouting.initialize("node-0", null, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE); - shardRouting = shardRouting.moveToStarted(); - CommonStats stats = new CommonStats(); - stats.fieldData = new FieldDataStats(); - stats.queryCache = new QueryCacheStats(); - stats.docs = new DocsStats(); - stats.store = new StoreStats(); - stats.indexing = new IndexingStats(); - stats.search = new SearchStats(); - stats.segments = new SegmentsStats(); - stats.merge = new MergeStats(); - stats.refresh = new RefreshStats(); - stats.completion = new CompletionStats(); - stats.requestCache = new RequestCacheStats(); - stats.get = new GetStats(); - stats.flush = new FlushStats(); - stats.warmer = new WarmerStats(); - shardStats.add(new ShardStats(shardRouting, new ShardPath(false, path, path, shardId), stats, null, null, null)); + ClusterIndexHealth indexHealth = indicesHealths.get(indexName); + IndexStats indexStats = indicesStats.get(indexName); + IndexMetaData indexMetaData = indicesMetaDatas.get(indexName); + + if (indexHealth != null) { + assertThat(row.get(0).value, equalTo(indexHealth.getStatus().toString().toLowerCase(Locale.ROOT))); + } else if (indexStats != null) { + assertThat(row.get(0).value, equalTo("red*")); + } else { + assertThat(row.get(0).value, equalTo("")); } - } - return IndicesStatsTests.newIndicesStatsResponse( - shardStats.toArray(new ShardStats[shardStats.size()]), shardStats.size(), shardStats.size(), 0, emptyList() - ); - } - private IndexMetaData[] removeRandomElement(IndexMetaData[] array) { - assert array != null; - assert array.length > 0; - final List collectionLessAnItem = new ArrayList<>(); - collectionLessAnItem.addAll(Arrays.asList(array)); - final int toRemoveIndex = randomIntBetween(0, array.length - 1); - collectionLessAnItem.remove(toRemoveIndex); - return collectionLessAnItem.toArray(new IndexMetaData[0]); + assertThat(row.get(1).value, equalTo(indexMetaData.getState().toString().toLowerCase(Locale.ROOT))); + assertThat(row.get(2).value, equalTo(indexName)); + assertThat(row.get(3).value, equalTo(indexMetaData.getIndexUUID())); + if (indexHealth != null) { + assertThat(row.get(4).value, equalTo(indexMetaData.getNumberOfShards())); + assertThat(row.get(5).value, equalTo(indexMetaData.getNumberOfReplicas())); + } else { + assertThat(row.get(4).value, nullValue()); + assertThat(row.get(5).value, nullValue()); + } + } } } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/14_cat_indices.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/14_cat_indices.yml index e4562d731ae75..ba50ac04bcb1f 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/14_cat_indices.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/14_cat_indices.yml @@ -44,6 +44,18 @@ setup: number_of_shards: "1" number_of_replicas: "0" + - do: + indices.create: + index: index3 + body: + settings: + number_of_shards: "1" + number_of_replicas: "0" + + - do: + indices.close: + index: index3 + --- teardown: - do: @@ -102,6 +114,58 @@ teardown: ) $/ +--- +"Test empty request while single authorized closed index": + + - do: + indices.create: + index: index_to_monitor + body: + settings: + number_of_shards: 1 + number_of_replicas: 1 + + - do: + index: + index: index_to_monitor + id: 0 + body: { foo: bar } + + - do: + index: + index: index_to_monitor + id: 1 + body: { foo: bar } + + - do: + index: + index: index_to_monitor + id: 2 + body: { foo: bar } + + - do: + indices.close: + index: index_to_monitor + + - do: + headers: { Authorization: "Basic Y2F0X3VzZXI6Y2F0X3Bhc3N3b3Jk" } # cat_user + cat.indices: {} + + - match: + $body: | + /^(yellow \s+ + close \s+ + index_to_monitor \s+ + ([a-zA-Z0-9=/_+]|[\\\-]){22} \s+ + 1 \s+ + 1 \s+ + \s+ + \s+ + \s+ + \s* + ) + $/ + --- "Test explicit request while multiple authorized indices": @@ -145,6 +209,67 @@ teardown: cat.indices: index: "this_*,index2" +--- +"Test explicit request while multiple opened/closed authorized indices": + + - do: + indices.create: + index: index_to_monitor + body: + settings: + number_of_shards: "1" + number_of_replicas: "0" + + - do: + indices.create: + index: this_index + body: + settings: + number_of_shards: "1" + number_of_replicas: "0" + + - do: + indices.create: + index: this_index_is_closed + body: + settings: + number_of_shards: "1" + number_of_replicas: "0" + + - do: + headers: { Authorization: "Basic Y2F0X3VzZXI6Y2F0X3Bhc3N3b3Jk" } # cat_user + cat.indices: + index: "this_index,index_to_monitor,this_index_is_closed" + v: false + h: i + + - match: + $body: | + /^(this_index \s*\n index_to_monitor \s*\n this_index_is_closed \n?) + |(this_index \s*\n this_index_is_closed \s*\n index_to_monitor \n?) + |(this_index_is_closed \s*\n this_index \s*\n index_to_monitor \n?) + |(this_index_is_closed \s*\n index_to_monitor \s*\n this_index \n?) + |(index_to_monitor \s*\n this_index \s*\n this_index_is_closed \n?) + |(index_to_monitor \s*\n this_index_is_closed \s*\n this_index \n?)$/ + + - do: + catch: forbidden + headers: { Authorization: "Basic Y2F0X3VzZXI6Y2F0X3Bhc3N3b3Jk" } # cat_user + cat.indices: + index: "index1,index_to_monitor" + + - do: + catch: forbidden + headers: { Authorization: "Basic Y2F0X3VzZXI6Y2F0X3Bhc3N3b3Jk" } # cat_user + cat.indices: + index: "this_*,index2" + + - do: + catch: forbidden + headers: { Authorization: "Basic Y2F0X3VzZXI6Y2F0X3Bhc3N3b3Jk" } # cat_user + cat.indices: + index: "this_index_is_closed,index2" + --- "Test wildcard request with multiple authorized indices":