-
Notifications
You must be signed in to change notification settings - Fork 25.6k
Retry shard movements during ESQL query #126653
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 8 commits
63b19d6
90f33b8
b1a04ff
545c50d
8b5f590
406df8d
1e3e92d
f7916f7
068b6be
a031270
1453368
4220622
2ea3292
782c595
8d2293e
54c0656
811d8fc
c20d53c
56128ee
9e2bb85
333ed60
b8b4a95
503f13e
e660001
0908649
b2819b6
e899a10
e298aaf
bb67124
ef0ffef
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,5 @@ | ||
| pr: 126653 | ||
| summary: Retry shard movements during ESQL query | ||
| area: ES|QL | ||
| type: enhancement | ||
| issues: [] |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,87 @@ | ||
| /* | ||
| * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one | ||
| * or more contributor license agreements. Licensed under the Elastic License | ||
| * 2.0; you may not use this file except in compliance with the Elastic License | ||
| * 2.0. | ||
| */ | ||
|
|
||
| package org.elasticsearch.xpack.esql.plugin; | ||
|
|
||
| import org.elasticsearch.action.index.IndexRequest; | ||
| import org.elasticsearch.action.support.WriteRequest; | ||
| import org.elasticsearch.cluster.metadata.IndexMetadata; | ||
| import org.elasticsearch.common.settings.Settings; | ||
| import org.elasticsearch.common.util.CollectionUtils; | ||
| import org.elasticsearch.compute.operator.exchange.ExchangeService; | ||
| import org.elasticsearch.plugins.Plugin; | ||
| import org.elasticsearch.test.transport.MockTransportService; | ||
| import org.elasticsearch.transport.TransportService; | ||
| import org.elasticsearch.xpack.esql.action.AbstractEsqlIntegTestCase; | ||
| import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; | ||
|
|
||
| import java.util.Collection; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
|
|
||
| import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; | ||
| import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; | ||
| import static org.elasticsearch.xpack.esql.EsqlTestUtils.getValuesList; | ||
| import static org.hamcrest.Matchers.hasSize; | ||
|
|
||
| public class DataNodeRequestSenderIT extends AbstractEsqlIntegTestCase { | ||
|
|
||
| @Override | ||
| protected Collection<Class<? extends Plugin>> nodePlugins() { | ||
| return CollectionUtils.appendToCopy(super.nodePlugins(), MockTransportService.TestPlugin.class); | ||
| } | ||
|
|
||
| public void testRetryOnShardMovement() { | ||
| internalCluster().ensureAtLeastNumDataNodes(2); | ||
|
|
||
| var index = randomIdentifier(); | ||
|
|
||
| assertAcked( | ||
| client().admin() | ||
| .indices() | ||
| .prepareCreate(index) | ||
| .setSettings( | ||
| Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) | ||
| ) | ||
| ); | ||
| client().prepareBulk(index) | ||
| .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) | ||
| .add(new IndexRequest().source("key", "value")) | ||
| .get(); | ||
|
|
||
| var shouldMove = new AtomicBoolean(true); | ||
|
|
||
| for (TransportService transportService : internalCluster().getInstances(TransportService.class)) { | ||
| as(transportService, MockTransportService.class).addRequestHandlingBehavior( | ||
| ExchangeService.OPEN_EXCHANGE_ACTION_NAME, | ||
| (handler, request, channel, task) -> { | ||
| // move index shard | ||
| if (shouldMove.compareAndSet(true, false)) { | ||
| var currentShardNodeId = clusterService().state() | ||
| .routingTable() | ||
| .index(index) | ||
| .shard(0) | ||
| .primaryShard() | ||
| .currentNodeId(); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I need to check if this could be replaced by a proper API call instead |
||
| assertAcked( | ||
| client().admin() | ||
| .indices() | ||
| .prepareUpdateSettings(index) | ||
| .setSettings(Settings.builder().put("index.routing.allocation.exclude._id", currentShardNodeId)) | ||
| ); | ||
| ensureGreen(index); | ||
| } | ||
| // execute data node request | ||
| handler.messageReceived(request, channel, task); | ||
| } | ||
| ); | ||
| } | ||
|
|
||
| try (EsqlQueryResponse resp = run("FROM " + index + "*")) { | ||
| assertThat(getValuesList(resp), hasSize(1)); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -23,14 +23,12 @@ | |||||||||||||||||||
| import org.elasticsearch.common.util.concurrent.ConcurrentCollections; | ||||||||||||||||||||
| import org.elasticsearch.compute.operator.DriverProfile; | ||||||||||||||||||||
| import org.elasticsearch.compute.operator.FailureCollector; | ||||||||||||||||||||
| import org.elasticsearch.core.TimeValue; | ||||||||||||||||||||
| import org.elasticsearch.index.Index; | ||||||||||||||||||||
| import org.elasticsearch.index.query.QueryBuilder; | ||||||||||||||||||||
| import org.elasticsearch.index.shard.ShardId; | ||||||||||||||||||||
| import org.elasticsearch.search.SearchShardTarget; | ||||||||||||||||||||
| import org.elasticsearch.search.internal.AliasFilter; | ||||||||||||||||||||
| import org.elasticsearch.tasks.CancellableTask; | ||||||||||||||||||||
| import org.elasticsearch.tasks.Task; | ||||||||||||||||||||
| import org.elasticsearch.tasks.TaskCancelledException; | ||||||||||||||||||||
| import org.elasticsearch.transport.TransportException; | ||||||||||||||||||||
| import org.elasticsearch.transport.TransportRequestOptions; | ||||||||||||||||||||
|
|
@@ -41,6 +39,7 @@ | |||||||||||||||||||
| import java.util.Collections; | ||||||||||||||||||||
| import java.util.Comparator; | ||||||||||||||||||||
| import java.util.HashMap; | ||||||||||||||||||||
| import java.util.HashSet; | ||||||||||||||||||||
| import java.util.IdentityHashMap; | ||||||||||||||||||||
| import java.util.Iterator; | ||||||||||||||||||||
| import java.util.LinkedHashMap; | ||||||||||||||||||||
|
|
@@ -53,6 +52,9 @@ | |||||||||||||||||||
| import java.util.concurrent.atomic.AtomicBoolean; | ||||||||||||||||||||
| import java.util.concurrent.atomic.AtomicInteger; | ||||||||||||||||||||
| import java.util.concurrent.locks.ReentrantLock; | ||||||||||||||||||||
| import java.util.function.Predicate; | ||||||||||||||||||||
|
|
||||||||||||||||||||
| import static org.elasticsearch.core.TimeValue.timeValueNanos; | ||||||||||||||||||||
|
|
||||||||||||||||||||
| /** | ||||||||||||||||||||
| * Handles computes within a single cluster by dispatching {@link DataNodeRequest} to data nodes | ||||||||||||||||||||
|
|
@@ -77,6 +79,7 @@ abstract class DataNodeRequestSender { | |||||||||||||||||||
| private final Executor esqlExecutor; | ||||||||||||||||||||
| private final String clusterAlias; | ||||||||||||||||||||
| private final CancellableTask rootTask; | ||||||||||||||||||||
| private final SearchShardsRequest searchShardsRequest; | ||||||||||||||||||||
| private final boolean allowPartialResults; | ||||||||||||||||||||
| private final Semaphore concurrentRequests; | ||||||||||||||||||||
| private final ReentrantLock sendingLock = new ReentrantLock(); | ||||||||||||||||||||
|
|
@@ -85,50 +88,50 @@ abstract class DataNodeRequestSender { | |||||||||||||||||||
| private final Map<ShardId, ShardFailure> shardFailures = ConcurrentCollections.newConcurrentMap(); | ||||||||||||||||||||
| private final AtomicInteger skippedShards = new AtomicInteger(); | ||||||||||||||||||||
| private final AtomicBoolean changed = new AtomicBoolean(); | ||||||||||||||||||||
| private final AtomicBoolean reResolvingUnavailableShards = new AtomicBoolean(false); | ||||||||||||||||||||
| private boolean reportedFailure = false; // guarded by sendingLock | ||||||||||||||||||||
|
|
||||||||||||||||||||
| DataNodeRequestSender( | ||||||||||||||||||||
| TransportService transportService, | ||||||||||||||||||||
| Executor esqlExecutor, | ||||||||||||||||||||
| String clusterAlias, | ||||||||||||||||||||
| CancellableTask rootTask, | ||||||||||||||||||||
| OriginalIndices originalIndices, | ||||||||||||||||||||
| QueryBuilder requestFilter, | ||||||||||||||||||||
| boolean allowPartialResults, | ||||||||||||||||||||
| int concurrentRequests | ||||||||||||||||||||
| ) { | ||||||||||||||||||||
| this.transportService = transportService; | ||||||||||||||||||||
| this.esqlExecutor = esqlExecutor; | ||||||||||||||||||||
| this.clusterAlias = clusterAlias; | ||||||||||||||||||||
| this.rootTask = rootTask; | ||||||||||||||||||||
| this.searchShardsRequest = new SearchShardsRequest( | ||||||||||||||||||||
| originalIndices.indices(), | ||||||||||||||||||||
| originalIndices.indicesOptions(), | ||||||||||||||||||||
| requestFilter, | ||||||||||||||||||||
| null, | ||||||||||||||||||||
| null, | ||||||||||||||||||||
| true, // unavailable_shards will be handled by the sender | ||||||||||||||||||||
| clusterAlias | ||||||||||||||||||||
| ); | ||||||||||||||||||||
| this.allowPartialResults = allowPartialResults; | ||||||||||||||||||||
| this.concurrentRequests = concurrentRequests > 0 ? new Semaphore(concurrentRequests) : null; | ||||||||||||||||||||
| } | ||||||||||||||||||||
|
|
||||||||||||||||||||
| final void startComputeOnDataNodes( | ||||||||||||||||||||
| String clusterAlias, | ||||||||||||||||||||
| Set<String> concreteIndices, | ||||||||||||||||||||
| OriginalIndices originalIndices, | ||||||||||||||||||||
| QueryBuilder requestFilter, | ||||||||||||||||||||
| Runnable runOnTaskFailure, | ||||||||||||||||||||
| ActionListener<ComputeResponse> listener | ||||||||||||||||||||
| ) { | ||||||||||||||||||||
| final void startComputeOnDataNodes(Set<String> concreteIndices, Runnable runOnTaskFailure, ActionListener<ComputeResponse> listener) { | ||||||||||||||||||||
| final long startTimeInNanos = System.nanoTime(); | ||||||||||||||||||||
| searchShards(rootTask, clusterAlias, requestFilter, concreteIndices, originalIndices, ActionListener.wrap(targetShards -> { | ||||||||||||||||||||
| searchShards(shardId -> concreteIndices.contains(shardId.getIndexName()), ActionListener.wrap(targetShards -> { | ||||||||||||||||||||
| try (var computeListener = new ComputeListener(transportService.getThreadPool(), runOnTaskFailure, listener.map(profiles -> { | ||||||||||||||||||||
| return new ComputeResponse( | ||||||||||||||||||||
| profiles, | ||||||||||||||||||||
| TimeValue.timeValueNanos(System.nanoTime() - startTimeInNanos), | ||||||||||||||||||||
| timeValueNanos(System.nanoTime() - startTimeInNanos), | ||||||||||||||||||||
| targetShards.totalShards(), | ||||||||||||||||||||
| targetShards.totalShards() - shardFailures.size() - skippedShards.get(), | ||||||||||||||||||||
| targetShards.skippedShards() + skippedShards.get(), | ||||||||||||||||||||
| shardFailures.size(), | ||||||||||||||||||||
| selectFailures() | ||||||||||||||||||||
| ); | ||||||||||||||||||||
| }))) { | ||||||||||||||||||||
| for (TargetShard shard : targetShards.shards.values()) { | ||||||||||||||||||||
| for (DiscoveryNode node : shard.remainingNodes) { | ||||||||||||||||||||
| nodePermits.putIfAbsent(node, new Semaphore(1)); | ||||||||||||||||||||
| } | ||||||||||||||||||||
| } | ||||||||||||||||||||
| pendingShardIds.addAll(order(targetShards)); | ||||||||||||||||||||
| trySendingRequestsForPendingShards(targetShards, computeListener); | ||||||||||||||||||||
| } | ||||||||||||||||||||
|
|
@@ -171,15 +174,17 @@ private void trySendingRequestsForPendingShards(TargetShards targetShards, Compu | |||||||||||||||||||
| if (changed.compareAndSet(true, false) == false) { | ||||||||||||||||||||
| break; | ||||||||||||||||||||
| } | ||||||||||||||||||||
| for (ShardId shardId : pendingShardIds) { | ||||||||||||||||||||
| if (targetShards.getShard(shardId).remainingNodes.isEmpty()) { | ||||||||||||||||||||
| shardFailures.compute( | ||||||||||||||||||||
| shardId, | ||||||||||||||||||||
| (k, v) -> new ShardFailure( | ||||||||||||||||||||
| true, | ||||||||||||||||||||
| v == null ? new NoShardAvailableActionException(shardId, "no shard copies found") : v.failure | ||||||||||||||||||||
| ) | ||||||||||||||||||||
| ); | ||||||||||||||||||||
| if (reResolvingUnavailableShards.get() == false) { | ||||||||||||||||||||
| for (ShardId shardId : pendingShardIds) { | ||||||||||||||||||||
| if (targetShards.getShard(shardId).remainingNodes.isEmpty()) { | ||||||||||||||||||||
| shardFailures.compute( | ||||||||||||||||||||
| shardId, | ||||||||||||||||||||
| (k, v) -> new ShardFailure( | ||||||||||||||||||||
| true, | ||||||||||||||||||||
| v == null ? new NoShardAvailableActionException(shardId, "no shard copies found") : v.failure | ||||||||||||||||||||
| ) | ||||||||||||||||||||
| ); | ||||||||||||||||||||
| } | ||||||||||||||||||||
| } | ||||||||||||||||||||
| } | ||||||||||||||||||||
| if (reportedFailure | ||||||||||||||||||||
|
|
@@ -238,11 +243,29 @@ private List<ShardSearchFailure> selectFailures() { | |||||||||||||||||||
| private void sendOneNodeRequest(TargetShards targetShards, ComputeListener computeListener, NodeRequest request) { | ||||||||||||||||||||
| final ActionListener<List<DriverProfile>> listener = computeListener.acquireCompute(); | ||||||||||||||||||||
| sendRequest(request.node, request.shardIds, request.aliasFilters, new NodeListener() { | ||||||||||||||||||||
|
|
||||||||||||||||||||
| private final Set<ShardId> pendingRetries = new HashSet<>(); | ||||||||||||||||||||
|
|
||||||||||||||||||||
| void onAfter(List<DriverProfile> profiles) { | ||||||||||||||||||||
| nodePermits.get(request.node).release(); | ||||||||||||||||||||
| if (concurrentRequests != null) { | ||||||||||||||||||||
| concurrentRequests.release(); | ||||||||||||||||||||
| } | ||||||||||||||||||||
|
|
||||||||||||||||||||
| // TODO limit attempts | ||||||||||||||||||||
| if (pendingRetries.isEmpty() == false) { | ||||||||||||||||||||
| reResolvingUnavailableShards.set(true); | ||||||||||||||||||||
| // TODO narrow down search resolution to pending shards only | ||||||||||||||||||||
| searchShards(pendingRetries::contains, computeListener.acquireAvoid().delegateFailure((l, newSearchShards) -> { | ||||||||||||||||||||
| for (var entry : newSearchShards.shards.entrySet()) { | ||||||||||||||||||||
| targetShards.shards.get(entry.getKey()).remainingNodes.addAll(entry.getValue().remainingNodes); | ||||||||||||||||||||
| } | ||||||||||||||||||||
| reResolvingUnavailableShards.set(false); | ||||||||||||||||||||
| trySendingRequestsForPendingShards(targetShards, computeListener); | ||||||||||||||||||||
| l.onResponse(null); | ||||||||||||||||||||
| })); | ||||||||||||||||||||
| } | ||||||||||||||||||||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The other branch should be in else? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not necessary. We could make progress without waiting for the moved shard(s) resolution in case there are other shards in queue. |
||||||||||||||||||||
|
|
||||||||||||||||||||
| trySendingRequestsForPendingShards(targetShards, computeListener); | ||||||||||||||||||||
| listener.onResponse(profiles); | ||||||||||||||||||||
| } | ||||||||||||||||||||
|
|
@@ -255,10 +278,11 @@ public void onResponse(DataNodeComputeResponse response) { | |||||||||||||||||||
| shardFailures.remove(shardId); | ||||||||||||||||||||
| } | ||||||||||||||||||||
| } | ||||||||||||||||||||
| for (Map.Entry<ShardId, Exception> e : response.shardLevelFailures().entrySet()) { | ||||||||||||||||||||
| final ShardId shardId = e.getKey(); | ||||||||||||||||||||
| trackShardLevelFailure(shardId, false, e.getValue()); | ||||||||||||||||||||
| for (var entry : response.shardLevelFailures().entrySet()) { | ||||||||||||||||||||
| final ShardId shardId = entry.getKey(); | ||||||||||||||||||||
| trackShardLevelFailure(shardId, false, entry.getValue()); | ||||||||||||||||||||
| pendingShardIds.add(shardId); | ||||||||||||||||||||
| maybeScheduleRetry(shardId, entry.getValue()); | ||||||||||||||||||||
| } | ||||||||||||||||||||
| onAfter(response.profiles()); | ||||||||||||||||||||
| } | ||||||||||||||||||||
|
|
@@ -267,7 +291,8 @@ public void onResponse(DataNodeComputeResponse response) { | |||||||||||||||||||
| public void onFailure(Exception e, boolean receivedData) { | ||||||||||||||||||||
| for (ShardId shardId : request.shardIds) { | ||||||||||||||||||||
| trackShardLevelFailure(shardId, receivedData, e); | ||||||||||||||||||||
| pendingShardIds.add(shardId); | ||||||||||||||||||||
| pendingShardIds.add(shardId);// TODO should this shard be added only in case of non-fatal failure? | ||||||||||||||||||||
| maybeScheduleRetry(shardId, e); | ||||||||||||||||||||
|
||||||||||||||||||||
| } | ||||||||||||||||||||
| onAfter(List.of()); | ||||||||||||||||||||
| } | ||||||||||||||||||||
|
|
@@ -281,6 +306,13 @@ public void onSkip() { | |||||||||||||||||||
| onResponse(new DataNodeComputeResponse(List.of(), Map.of())); | ||||||||||||||||||||
| } | ||||||||||||||||||||
| } | ||||||||||||||||||||
|
|
||||||||||||||||||||
| private void maybeScheduleRetry(ShardId shardId, Exception e) { | ||||||||||||||||||||
| if (targetShards.getShard(shardId).remainingNodes.isEmpty() | ||||||||||||||||||||
| && unwrapFailure(e) instanceof NoShardAvailableActionException) { | ||||||||||||||||||||
|
||||||||||||||||||||
| public static boolean isShardNotAvailableException(final Throwable e) { | |
| final Throwable actual = ExceptionsHelper.unwrapCause(e); | |
| return (actual instanceof ShardNotFoundException | |
| || actual instanceof IndexNotFoundException | |
| || actual instanceof IllegalIndexShardStateException | |
| || actual instanceof NoShardAvailableActionException | |
| || actual instanceof UnavailableShardsException | |
| || actual instanceof AlreadyClosedException); | |
| } |
Probably this should only retry ShardNotFoundException/NoShardAvailableActionException/UnavailableShardsException. Not sure. Please let me know what do you think.
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder if we should make the logic that resolves the new target nodes a helper method and call it under sendingLock to avoid handling concurrency.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we make this test similar to SearchWhileRelocatingIT? We continue running ES|QL on one thread while moving shards back and forth between two sets of nodes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added