Skip to content

Commit 638d3c2

Browse files
authored
Change exception type when timing out waiting for specific seqno in fleet search api. (#114526) (#114645)
Without this change request fails with `ElasticsearchTimeoutException` if waiting for seqno times out. This results in a 500 status code. With this change the `SearchTimeoutException` is used which results in a 504 status code. This is a more appropriate response code for time-outs. Closes #114395
1 parent 2d8ea99 commit 638d3c2

File tree

1 file changed

+8
-3
lines changed

1 file changed

+8
-3
lines changed

server/src/main/java/org/elasticsearch/search/SearchService.java

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@
1717
import org.apache.lucene.search.Query;
1818
import org.apache.lucene.search.TopDocs;
1919
import org.elasticsearch.ElasticsearchException;
20-
import org.elasticsearch.ElasticsearchTimeoutException;
2120
import org.elasticsearch.action.ActionListener;
2221
import org.elasticsearch.action.ActionRunnable;
2322
import org.elasticsearch.action.ResolvedIndices;
@@ -34,6 +33,7 @@
3433
import org.elasticsearch.common.UUIDs;
3534
import org.elasticsearch.common.breaker.CircuitBreaker;
3635
import org.elasticsearch.common.component.AbstractLifecycleComponent;
36+
import org.elasticsearch.common.logging.LoggerMessageFormat;
3737
import org.elasticsearch.common.lucene.Lucene;
3838
import org.elasticsearch.common.settings.Setting;
3939
import org.elasticsearch.common.settings.Setting.Property;
@@ -114,6 +114,7 @@
114114
import org.elasticsearch.search.query.QuerySearchRequest;
115115
import org.elasticsearch.search.query.QuerySearchResult;
116116
import org.elasticsearch.search.query.ScrollQuerySearchResult;
117+
import org.elasticsearch.search.query.SearchTimeoutException;
117118
import org.elasticsearch.search.rank.feature.RankFeatureResult;
118119
import org.elasticsearch.search.rank.feature.RankFeatureShardPhase;
119120
import org.elasticsearch.search.rank.feature.RankFeatureShardRequest;
@@ -597,9 +598,13 @@ private <T extends RefCounted> void ensureAfterSeqNoRefreshed(
597598
final TimeValue timeout = request.getWaitForCheckpointsTimeout();
598599
final Scheduler.ScheduledCancellable timeoutTask = NO_TIMEOUT.equals(timeout) ? null : threadPool.schedule(() -> {
599600
if (isDone.compareAndSet(false, true)) {
600-
listener.onFailure(
601-
new ElasticsearchTimeoutException("Wait for seq_no [{}] refreshed timed out [{}]", waitForCheckpoint, timeout)
601+
var shardTarget = new SearchShardTarget(
602+
shard.routingEntry().currentNodeId(),
603+
shard.shardId(),
604+
request.getClusterAlias()
602605
);
606+
var message = LoggerMessageFormat.format("Wait for seq_no [{}] refreshed timed out [{}]", waitForCheckpoint, timeout);
607+
listener.onFailure(new SearchTimeoutException(shardTarget, message));
603608
}
604609
}, timeout, EsExecutors.DIRECT_EXECUTOR_SERVICE);
605610

0 commit comments

Comments
 (0)