Fix indices shown in _cat/indices (#43286)

After two recent changes (#38824 and #33888), the _cat/indices API
no longer report information for active recovering indices and
non-replicated closed indices. It also misreport replicated closed
indices that are potentially not authorized for the user.

This commit changes how the cat action works by first using the
Get Settings API in order to resolve authorized indices. It then uses
the Cluster State, Cluster Health and Indices Stats APIs to retrieve
 information about the indices.

Closes #39933
This commit is contained in:
Tanguy Leroux 2019-06-25 20:02:34 +02:00 committed by GitHub
parent 126c2fd2d5
commit 0dc1c12f13
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 494 additions and 249 deletions

View File

@ -666,7 +666,7 @@ public class ActionModule extends AbstractModule {
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));

View File

@ -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 @@ public class RestIndicesAction extends AbstractCatAction {
@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);
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 -> client.admin().cluster().state(clusterStateRequest, new RestActionListener<ClusterStateResponse>(channel) {
return channel -> {
final ActionListener<Table> listener = ActionListener.notifyOnce(new RestResponseListener<Table>(channel) {
@Override
public RestResponse buildResponse(final Table table) throws Exception {
return RestTable.buildResponse(table, channel);
}
});
sendGetSettingsRequest(indices, indicesOptions, local, masterNodeTimeout, client, new ActionListener<GetSettingsResponse>() {
@Override
public void onResponse(final GetSettingsResponse getSettingsResponse) {
final GroupedActionListener<ActionResponse> 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<GetSettingsResponse> 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<ClusterStateResponse> 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<ClusterHealthResponse> 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<IndicesStatsResponse> 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<ActionResponse> createGroupedListener(final RestRequest request, final int size,
final ActionListener<Table> listener) {
return new GroupedActionListener<>(new ActionListener<Collection<ActionResponse>>() {
@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()));
public void onResponse(final Collection<ActionResponse> responses) {
GetSettingsResponse settingsResponse = extractResponse(responses, GetSettingsResponse.class);
Map<String, Settings> indicesSettings = StreamSupport.stream(settingsResponse.getIndexToSettings().spliterator(), false)
.collect(Collectors.toMap(cursor -> cursor.key, cursor -> cursor.value));
client.admin().cluster().health(clusterHealthRequest, new RestActionListener<ClusterHealthResponse>(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));
ClusterStateResponse stateResponse = extractResponse(responses, ClusterStateResponse.class);
Map<String, IndexMetaData> indicesStates = StreamSupport.stream(stateResponse.getState().getMetaData().spliterator(), false)
.collect(Collectors.toMap(indexMetaData -> indexMetaData.getIndex().getName(), Function.identity()));
client.admin().indices().stats(indicesStatsRequest, new RestResponseListener<IndicesStatsResponse>(channel) {
@Override
public RestResponse buildResponse(IndicesStatsResponse indicesStatsResponse) throws Exception {
final Table tab = buildTable(request, indicesMetaData, clusterHealthResponse, indicesStatsResponse);
return RestTable.buildResponse(tab, channel);
}
});
}
});
ClusterHealthResponse healthResponse = extractResponse(responses, ClusterHealthResponse.class);
Map<String, ClusterIndexHealth> indicesHealths = healthResponse.getIndices();
IndicesStatsResponse statsResponse = extractResponse(responses, IndicesStatsResponse.class);
Map<String, IndexStats> 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<String> RESPONSE_PARAMS;
@ -393,18 +494,35 @@ public class RestIndicesAction extends AbstractCatAction {
// 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<String, Settings> indicesSettings,
final Map<String, ClusterIndexHealth> indicesHealths,
final Map<String, IndexStats> indicesStats,
final Map<String, IndexMetaData> 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 @@ public class RestIndicesAction extends AbstractCatAction {
// 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 @@ public class RestIndicesAction extends AbstractCatAction {
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<String, IndexMetaData> 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 extends ActionResponse> A extractResponse(final Collection<? extends ActionResponse> responses, Class<A> c) {
return (A) responses.stream().filter(c::isInstance).findFirst().get();
}
}

View File

@ -27,6 +27,8 @@ import org.elasticsearch.action.admin.indices.stats.CommonStats;
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.IndexTemplateMetaData;
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.ByteSizeUnit;
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.Locale;
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 class TransportRolloverActionTests extends ESTestCase {
.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<String, Boolean> results2 = evaluateConditions(conditions, null, indicesStats);
assertThat(results2.size(), equalTo(3));
results2.forEach((k, v) -> assertFalse(v));
@ -490,4 +512,42 @@ public class TransportRolloverActionTests extends ESTestCase {
when(condition.evaluate(any())).thenReturn(new Condition.Result(condition, true));
return condition;
}
public static IndicesStatsResponse randomIndicesStatsResponse(final IndexMetaData[] indices) {
List<ShardStats> 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()
);
}
}

View File

@ -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];
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;
}
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;
}
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;
}
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());
final int numIndices = randomIntBetween(3, 20);
final Map<String, Settings> indicesSettings = new LinkedHashMap<>();
final Map<String, IndexMetaData> indicesMetaDatas = new LinkedHashMap<>();
final Map<String, ClusterIndexHealth> indicesHealths = new LinkedHashMap<>();
final Map<String, IndexStats> indicesStats = new LinkedHashMap<>();
final IndexMetaData[] generatedIndicesMetaData = buildRandomIndicesMetaData(randomIntBetween(1, 5));
final ClusterState clusterState = buildClusterState(generatedIndicesMetaData);
final ClusterHealthResponse clusterHealthResponse = buildClusterHealthResponse(clusterState, generatedIndicesMetaData);
for (int i = 0; i < numIndices; i++) {
String indexName = "index-" + i;
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));
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);
// now, verify the table is correct
int count = 0;
List<Table.Cell> 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"));
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);
List<List<Table.Cell>> 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<Table.Cell> 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()));
}
}
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()));
public static IndicesStatsResponse randomIndicesStatsResponse(final IndexMetaData[] indices) {
List<ShardStats> 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));
if (frequently()) {
IndexStats indexStats = mock(IndexStats.class);
when(indexStats.getPrimaries()).thenReturn(new CommonStats());
when(indexStats.getTotal()).thenReturn(new CommonStats());
indicesStats.put(indexName, indexStats);
}
}
}
}
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<IndexMetaData> 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]);
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
List<Table.Cell> headers = table.getHeaders();
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<List<Table.Cell>> rows = table.getRows();
assertThat(rows.size(), equalTo(indicesMetaDatas.size()));
for (final List<Table.Cell> row : rows) {
final String indexName = (String) row.get(2).value;
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(""));
}
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());
}
}
}
}

View File

@ -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":