Skip to content

Commit

Permalink
Fix alias resolution runtime complexity. (#40263) (#40788)
Browse files Browse the repository at this point in the history
A user reported that the same query that takes ~900ms when querying an index
pattern only takes ~50ms when only querying indices that have matches. The
query is a date range query and we confirmed that the `can_match` phase works
as expected. I was able to reproduce this issue locally with a single node: with
900 1-shard indices, a query to an index pattern that matches all indices runs
in ~90ms while a query to the only index that has matches runs in 0-1ms.

This ended up not being related to the `can_match` phase but to the cost of
resolving aliases when querying an index pattern that matches lots of indices.
In that case, we first resolve the index pattern to a list of concrete indices
and then for each concrete index, we check whether it was matched through an
alias, meaning we might have to apply alias filters. Unfortunately this second
per-index operation runs in linear time with the number of matched concrete
indices, which means that alias resolution runs in O(num_indices^2) overall.
So queries get exponentially slower as an index pattern matches more indices.

I reorganized alias resolution into a one-step operation that runs in linear
time with the number of matches indices, and then a per-index operation that
runs in linear time with the number of aliases of this index. This makes alias
resolution run is O(num_indices * num_aliases_per_index) overall instead. When
testing the scenario described above, the `took` went down from ~90ms to ~10ms.
It is still more than the 0-1ms latency that one gets when only querying the
single index that has data, but still much better than what we had before.

Closes #40248
  • Loading branch information
jpountz committed Apr 4, 2019
1 parent 65cca2e commit 670e766
Show file tree
Hide file tree
Showing 11 changed files with 217 additions and 66 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -64,10 +64,9 @@
- length: { shards: 1 }
- match: { shards.0.0.index: test_index }
- match: { indices.test_index.aliases: [test_alias_filter_1, test_alias_filter_2]}
- match: { indices.test_index.filter.bool.should.0.term.field.value: value1 }
- length: { indices.test_index.filter.bool.should: 2 }
- lte: { indices.test_index.filter.bool.should.0.term.field.boost: 1.0 }
- gte: { indices.test_index.filter.bool.should.0.term.field.boost: 1.0 }
- match: { indices.test_index.filter.bool.should.1.term.field.value: value2}
- lte: { indices.test_index.filter.bool.should.1.term.field.boost: 1.0 }
- gte: { indices.test_index.filter.bool.should.1.term.field.boost: 1.0 }
- match: { indices.test_index.filter.bool.adjust_pure_negative: true}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,10 +88,11 @@ protected void masterOperation(final ClusterSearchShardsRequest request, final C
String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(clusterState, request);
Map<String, Set<String>> routingMap = indexNameExpressionResolver.resolveSearchRouting(state, request.routing(), request.indices());
Map<String, AliasFilter> indicesAndFilters = new HashMap<>();
Set<String> indicesAndAliases = indexNameExpressionResolver.resolveExpressions(clusterState, request.indices());
for (String index : concreteIndices) {
final AliasFilter aliasFilter = indicesService.buildAliasFilter(clusterState, index, request.indices());
final AliasFilter aliasFilter = indicesService.buildAliasFilter(clusterState, index, indicesAndAliases);
final String[] aliases = indexNameExpressionResolver.indexAliases(clusterState, index, aliasMetadata -> true, true,
request.indices());
indicesAndAliases);
indicesAndFilters.put(index, new AliasFilter(aliasFilter.getQueryBuilder(), aliases));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,8 +115,9 @@ protected void doExecute(Task task, ValidateQueryRequest request, ActionListener

@Override
protected ShardValidateQueryRequest newShardRequest(int numShards, ShardRouting shard, ValidateQueryRequest request) {
final AliasFilter aliasFilter = searchService.buildAliasFilter(clusterService.state(), shard.getIndexName(),
request.indices());
final ClusterState clusterState = clusterService.state();
final Set<String> indicesAndAliases = indexNameExpressionResolver.resolveExpressions(clusterState, request.indices());
final AliasFilter aliasFilter = searchService.buildAliasFilter(clusterState, shard.getIndexName(), indicesAndAliases);
return new ShardValidateQueryRequest(shard.shardId(), aliasFilter, request);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
import org.elasticsearch.transport.TransportService;

import java.io.IOException;
import java.util.Set;

/**
* Explain transport action. Computes the explain on the targeted shard.
Expand Down Expand Up @@ -83,8 +84,8 @@ protected boolean resolveIndex(ExplainRequest request) {

@Override
protected void resolveRequest(ClusterState state, InternalRequest request) {
final AliasFilter aliasFilter = searchService.buildAliasFilter(state, request.concreteIndex(),
request.request().index());
final Set<String> indicesAndAliases = indexNameExpressionResolver.resolveExpressions(state, request.request().index());
final AliasFilter aliasFilter = searchService.buildAliasFilter(state, request.concreteIndex(), indicesAndAliases);
request.request().filteringAlias(aliasFilter);
// Fail fast on the node that received the request.
if (request.request().routing() == null && state.getMetaData().routingRequired(request.concreteIndex())) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,9 +116,10 @@ public TransportSearchAction(ThreadPool threadPool, TransportService transportSe
private Map<String, AliasFilter> buildPerIndexAliasFilter(SearchRequest request, ClusterState clusterState,
Index[] concreteIndices, Map<String, AliasFilter> remoteAliasMap) {
final Map<String, AliasFilter> aliasFilterMap = new HashMap<>();
final Set<String> indicesAndAliases = indexNameExpressionResolver.resolveExpressions(clusterState, request.indices());
for (Index index : concreteIndices) {
clusterState.blocks().indexBlockedRaiseException(ClusterBlockLevel.READ, index.getName());
AliasFilter aliasFilter = searchService.buildAliasFilter(clusterState, index.getName(), request.indices());
AliasFilter aliasFilter = searchService.buildAliasFilter(clusterState, index.getName(), indicesAndAliases);
assert aliasFilter != null;
aliasFilterMap.put(index.getUUID(), aliasFilter);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.time.DateFormatter;
Expand All @@ -42,15 +43,19 @@
import java.time.ZoneOffset;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.SortedMap;
import java.util.Spliterators;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;

import static java.util.Collections.unmodifiableList;

Expand Down Expand Up @@ -322,71 +327,88 @@ public String resolveDateMathExpression(String dateExpression) {
return dateMathExpressionResolver.resolveExpression(dateExpression, new Context(null, null));
}

/**
* Resolve an array of expressions to the set of indices and aliases that these expressions match.
*/
public Set<String> resolveExpressions(ClusterState state, String... expressions) {
Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true, false);
List<String> resolvedExpressions = Arrays.asList(expressions);
for (ExpressionResolver expressionResolver : expressionResolvers) {
resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions);
}
return Collections.unmodifiableSet(new HashSet<>(resolvedExpressions));
}

/**
* Iterates through the list of indices and selects the effective list of filtering aliases for the
* given index.
* <p>Only aliases with filters are returned. If the indices list contains a non-filtering reference to
* the index itself - null is returned. Returns {@code null} if no filtering is required.
* <b>NOTE</b>: The provided expressions must have been resolved already via {@link #resolveExpressions}.
*/
public String[] filteringAliases(ClusterState state, String index, String... expressions) {
return indexAliases(state, index, AliasMetaData::filteringRequired, false, expressions);
public String[] filteringAliases(ClusterState state, String index, Set<String> resolvedExpressions) {
return indexAliases(state, index, AliasMetaData::filteringRequired, false, resolvedExpressions);
}

/**
* Whether to generate the candidate set from index aliases, or from the set of resolved expressions.
* @param indexAliasesSize the number of aliases of the index
* @param resolvedExpressionsSize the number of resolved expressions
*/
// pkg-private for testing
boolean iterateIndexAliases(int indexAliasesSize, int resolvedExpressionsSize) {
return indexAliasesSize <= resolvedExpressionsSize;
}

/**
* Iterates through the list of indices and selects the effective list of required aliases for the given index.
* <p>Only aliases where the given predicate tests successfully are returned. If the indices list contains a non-required reference to
* the index itself - null is returned. Returns {@code null} if no filtering is required.
* <p><b>NOTE</b>: the provided expressions must have been resolved already via {@link #resolveExpressions}.
*/
public String[] indexAliases(ClusterState state, String index, Predicate<AliasMetaData> requiredAlias, boolean skipIdentity,
String... expressions) {
// expand the aliases wildcard
List<String> resolvedExpressions = expressions != null ? Arrays.asList(expressions) : Collections.emptyList();
Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true, false);
for (ExpressionResolver expressionResolver : expressionResolvers) {
resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions);
}

Set<String> resolvedExpressions) {
if (isAllIndices(resolvedExpressions)) {
return null;
}

final IndexMetaData indexMetaData = state.metaData().getIndices().get(index);
if (indexMetaData == null) {
// Shouldn't happen
throw new IndexNotFoundException(index);
}
// optimize for the most common single index/alias scenario
if (resolvedExpressions.size() == 1) {
String alias = resolvedExpressions.get(0);

AliasMetaData aliasMetaData = indexMetaData.getAliases().get(alias);
if (aliasMetaData == null || requiredAlias.test(aliasMetaData) == false) {
return null;
}
return new String[]{alias};
if (skipIdentity == false && resolvedExpressions.contains(index)) {
return null;
}

final ImmutableOpenMap<String, AliasMetaData> indexAliases = indexMetaData.getAliases();
final AliasMetaData[] aliasCandidates;
if (iterateIndexAliases(indexAliases.size(), resolvedExpressions.size())) {
// faster to iterate indexAliases
aliasCandidates = StreamSupport.stream(Spliterators.spliteratorUnknownSize(indexAliases.values().iterator(), 0), false)
.map(cursor -> cursor.value)
.filter(aliasMetaData -> resolvedExpressions.contains(aliasMetaData.alias()))
.toArray(AliasMetaData[]::new);
} else {
// faster to iterate resolvedExpressions
aliasCandidates = resolvedExpressions.stream()
.map(indexAliases::get)
.filter(Objects::nonNull)
.toArray(AliasMetaData[]::new);
}

List<String> aliases = null;
for (String alias : resolvedExpressions) {
if (alias.equals(index)) {
if (skipIdentity) {
continue;
} else {
return null;
}
}
AliasMetaData aliasMetaData = indexMetaData.getAliases().get(alias);
// Check that this is an alias for the current index
// Otherwise - skip it
if (aliasMetaData != null) {
if (requiredAlias.test(aliasMetaData)) {
// If required - add it to the list of aliases
if (aliases == null) {
aliases = new ArrayList<>();
}
aliases.add(alias);
} else {
// If not, we have a non required alias for this index - no further checking needed
return null;
for (AliasMetaData aliasMetaData : aliasCandidates) {
if (requiredAlias.test(aliasMetaData)) {
// If required - add it to the list of aliases
if (aliases == null) {
aliases = new ArrayList<>();
}
aliases.add(aliasMetaData.alias());
} else {
// If not, we have a non required alias for this index - no further checking needed
return null;
}
}
if (aliases == null) {
Expand Down Expand Up @@ -513,7 +535,7 @@ public Map<String, Set<String>> resolveSearchRoutingAllIndices(MetaData metaData
* @param aliasesOrIndices the array containing index names
* @return true if the provided array maps to all indices, false otherwise
*/
public static boolean isAllIndices(List<String> aliasesOrIndices) {
public static boolean isAllIndices(Collection<String> aliasesOrIndices) {
return aliasesOrIndices == null || aliasesOrIndices.isEmpty() || isExplicitAllPattern(aliasesOrIndices);
}

Expand All @@ -524,8 +546,8 @@ public static boolean isAllIndices(List<String> aliasesOrIndices) {
* @param aliasesOrIndices the array containing index names
* @return true if the provided array explicitly maps to all indices, false otherwise
*/
static boolean isExplicitAllPattern(List<String> aliasesOrIndices) {
return aliasesOrIndices != null && aliasesOrIndices.size() == 1 && MetaData.ALL.equals(aliasesOrIndices.get(0));
static boolean isExplicitAllPattern(Collection<String> aliasesOrIndices) {
return aliasesOrIndices != null && aliasesOrIndices.size() == 1 && MetaData.ALL.equals(aliasesOrIndices.iterator().next());
}

/**
Expand Down Expand Up @@ -598,7 +620,7 @@ public long getStartTime() {
/**
* This is used to prevent resolving aliases to concrete indices but this also means
* that we might return aliases that point to a closed index. This is currently only used
* by {@link #filteringAliases(ClusterState, String, String...)} since it's the only one that needs aliases
* by {@link #filteringAliases(ClusterState, String, Set)} since it's the only one that needs aliases
*/
boolean isPreserveAliases() {
return preserveAliases;
Expand Down Expand Up @@ -642,6 +664,8 @@ public List<String> resolve(Context context, List<String> expressions) {
return resolveEmptyOrTrivialWildcard(options, metaData);
}

// TODO: Fix API to work with sets rather than lists since we need to convert to sets
// internally anyway.
Set<String> result = innerResolve(context, expressions, options, metaData);

if (result == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1385,7 +1385,7 @@ interface IndexDeletionAllowedPredicate {
(Index index, IndexSettings indexSettings) -> canDeleteIndexContents(index, indexSettings);
private final IndexDeletionAllowedPredicate ALWAYS_TRUE = (Index index, IndexSettings indexSettings) -> true;

public AliasFilter buildAliasFilter(ClusterState state, String index, String... expressions) {
public AliasFilter buildAliasFilter(ClusterState state, String index, Set<String> resolvedExpressions) {
/* Being static, parseAliasFilter doesn't have access to whatever guts it needs to parse a query. Instead of passing in a bunch
* of dependencies we pass in a function that can perform the parsing. */
CheckedFunction<byte[], QueryBuilder, IOException> filterParser = bytes -> {
Expand All @@ -1394,8 +1394,8 @@ public AliasFilter buildAliasFilter(ClusterState state, String index, String...
return parseInnerQueryBuilder(parser);
}
};
String[] aliases = indexNameExpressionResolver.filteringAliases(state, index, expressions);
IndexMetaData indexMetaData = state.metaData().index(index);
String[] aliases = indexNameExpressionResolver.filteringAliases(state, index, resolvedExpressions);
return new AliasFilter(ShardSearchRequest.parseAliasFilter(filterParser, indexMetaData, aliases), aliases);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,7 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicInteger;
Expand Down Expand Up @@ -1004,8 +1005,8 @@ public void run() {
}
}

public AliasFilter buildAliasFilter(ClusterState state, String index, String... expressions) {
return indicesService.buildAliasFilter(state, index, expressions);
public AliasFilter buildAliasFilter(ClusterState state, String index, Set<String> resolvedExpressions) {
return indicesService.buildAliasFilter(state, index, resolvedExpressions);
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.elasticsearch.cluster.metadata;

public class IndexNameExpressionResolverAliasIterationTests extends IndexNameExpressionResolverTests {

protected IndexNameExpressionResolver getIndexNameExpressionResolver() {
return new IndexNameExpressionResolver() {
@Override
boolean iterateIndexAliases(int indexAliasesSize, int resolvedExpressionsSize) {
return true;
}
};
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.elasticsearch.cluster.metadata;

public class IndexNameExpressionResolverExpressionsIterationTests extends IndexNameExpressionResolverTests {

protected IndexNameExpressionResolver getIndexNameExpressionResolver() {
return new IndexNameExpressionResolver() {
@Override
boolean iterateIndexAliases(int indexAliasesSize, int resolvedExpressionsSize) {
return false;
}
};
}

}
Loading

0 comments on commit 670e766

Please sign in to comment.