Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions docs/changelog/133188.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
pr: 133188
summary: Don't fail search if bottom doc can't be formatted
area: Search
type: bug
issues:
- 125321
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@

package org.elasticsearch.action.search;

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.search.TopFieldDocs;
import org.elasticsearch.action.ActionListener;
Expand All @@ -25,13 +26,15 @@
import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.transport.Transport;

import java.util.Arrays;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.function.BiFunction;

import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize;

class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<SearchPhaseResult> {
private static final Logger logger = LogManager.getLogger(SearchQueryThenFetchAsyncAction.class);

private final SearchProgressListener progressListener;

Expand Down Expand Up @@ -121,7 +124,17 @@ && getRequest().scroll() == null
}
}
}
bottomSortCollector.consumeTopDocs(topDocs, queryResult.sortValueFormats());
try {
bottomSortCollector.consumeTopDocs(topDocs, queryResult.sortValueFormats());
} catch (Exception e) {
// In case the collecting fails, e.g. because of a formatting error, we log the error and continue
logger.debug(
"failed to consume top docs for shard [{}] with sort fields [{}]: {}",
result.getShardIndex(),
Arrays.toString(topDocs.fields),
e
);
}
}
super.onShardResult(result, shardIt);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TopFieldDocs;
import org.apache.lucene.search.TotalHits;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.OriginalIndices;
Expand All @@ -27,6 +28,7 @@
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.NoopCircuitBreaker;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.core.TimeValue;
Expand All @@ -50,7 +52,9 @@
import org.elasticsearch.test.VersionUtils;
import org.elasticsearch.test.index.IndexVersionUtils;
import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportService;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
Expand All @@ -59,13 +63,16 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.LongSupplier;

import static java.util.Collections.singletonList;
import static org.elasticsearch.test.VersionUtils.allVersions;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.instanceOf;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;

public class SearchQueryThenFetchAsyncActionTests extends ESTestCase {
public void testBottomFieldSort() throws Exception {
Expand Down Expand Up @@ -740,4 +747,190 @@ public void run() {
assertThat(e.getMessage(), equalTo("One of the shards is incompatible with the required minimum version [" + minVersion + "]"));
}
}

static class BadRawDocValueFormat implements DocValueFormat {
@Override
public String getWriteableName() {
return "bad";
}

@Override
public void writeTo(StreamOutput out) throws IOException {}

@Override
public Object format(long value) {
if (value == Long.MAX_VALUE) {
// Simulate a bad value that cannot be formatted correctly
throw new IllegalArgumentException("Cannot format Long.MAX_VALUE");
}
return RawDocValueFormat.INSTANCE.format(value);
}

@Override
public Object format(double value) {
return RawDocValueFormat.INSTANCE.format(value);
}

@Override
public Object format(BytesRef value) {
return RawDocValueFormat.INSTANCE.format(value);
}

@Override
public long parseLong(String value, boolean roundUp, LongSupplier now) {
return RawDocValueFormat.INSTANCE.parseLong(value, roundUp, now);
}

@Override
public double parseDouble(String value, boolean roundUp, LongSupplier now) {
return RawDocValueFormat.INSTANCE.parseLong(value, roundUp, now);
}

@Override
public BytesRef parseBytesRef(Object value) {
return RawDocValueFormat.INSTANCE.parseBytesRef(value);
}

@Override
public Object formatSortValue(Object value) {
return RawDocValueFormat.INSTANCE.formatSortValue(value);
}
}

// Test what happens if doc formatter fails to format the bottom sort values
public void testBadFormatting() throws Exception {
final TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(
0,
System.nanoTime(),
System::nanoTime
);

Map<String, Transport.Connection> lookup = new ConcurrentHashMap<>();
DiscoveryNode primaryNode = DiscoveryNodeUtils.create("node1");
DiscoveryNode replicaNode = DiscoveryNodeUtils.create("node2");
lookup.put("node1", new SearchAsyncActionTests.MockConnection(primaryNode));
lookup.put("node2", new SearchAsyncActionTests.MockConnection(replicaNode));

int numShards = randomIntBetween(10, 20);
int numConcurrent = randomIntBetween(1, 4);
AtomicInteger numWithTopDocs = new AtomicInteger();
AtomicInteger successfulOps = new AtomicInteger();
AtomicBoolean canReturnNullResponse = new AtomicBoolean(false);
var transportService = mock(TransportService.class);
when(transportService.getLocalNode()).thenReturn(primaryNode);
SearchTransportService searchTransportService = new SearchTransportService(transportService, null, null) {
@Override
public void sendExecuteQuery(
Transport.Connection connection,
ShardSearchRequest request,
SearchTask task,
ActionListener<SearchPhaseResult> listener
) {
int shardId = request.shardId().id();
if (request.canReturnNullResponseIfMatchNoDocs()) {
canReturnNullResponse.set(true);
}
if (request.getBottomSortValues() != null) {
numWithTopDocs.incrementAndGet();
}
QuerySearchResult queryResult = new QuerySearchResult(
new ShardSearchContextId("N/A", 123),
new SearchShardTarget("node1", new ShardId("idx", "na", shardId), null),
null
);
try {
SortField sortField = new SortField("RegistrationDate", SortField.Type.LONG);
queryResult.topDocs(
new TopDocsAndMaxScore(
new TopFieldDocs(
new TotalHits(1, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO),
new FieldDoc[] { new FieldDoc(0, Float.NaN, new Object[] { Long.MAX_VALUE }) },
new SortField[] { sortField }
),
Float.NaN
),
new DocValueFormat[] { new BadRawDocValueFormat() }
);
queryResult.from(0);
queryResult.size(1);
successfulOps.incrementAndGet();
queryResult.incRef();
new Thread(() -> ActionListener.respondAndRelease(listener, queryResult)).start();
} finally {
queryResult.decRef();
}
}
};
CountDownLatch latch = new CountDownLatch(1);
GroupShardsIterator<SearchShardIterator> shardsIter = SearchAsyncActionTests.getShardsIter(
"idx",
new OriginalIndices(new String[] { "idx" }, SearchRequest.DEFAULT_INDICES_OPTIONS),
numShards,
randomBoolean(),
primaryNode,
replicaNode
);
final SearchRequest searchRequest = new SearchRequest();
searchRequest.setMaxConcurrentShardRequests(numConcurrent);
searchRequest.setBatchedReduceSize(2);
searchRequest.source(new SearchSourceBuilder().size(1).sort(SortBuilders.fieldSort("timestamp")));
searchRequest.source().trackTotalHitsUpTo(2);
searchRequest.allowPartialSearchResults(false);
SearchPhaseController controller = new SearchPhaseController((t, r) -> InternalAggregationTestCase.emptyReduceContextBuilder());
SearchTask task = new SearchTask(0, "n/a", "n/a", () -> "test", null, Collections.emptyMap());
try (
QueryPhaseResultConsumer resultConsumer = new QueryPhaseResultConsumer(
searchRequest,
EsExecutors.DIRECT_EXECUTOR_SERVICE,
new NoopCircuitBreaker(CircuitBreaker.REQUEST),
controller,
task::isCancelled,
task.getProgressListener(),
shardsIter.size(),
exc -> {}
)
) {
SearchQueryThenFetchAsyncAction action = new SearchQueryThenFetchAsyncAction(
logger,
null,
searchTransportService,
(clusterAlias, node) -> lookup.get(node),
Collections.singletonMap("_na_", AliasFilter.EMPTY),
Collections.emptyMap(),
EsExecutors.DIRECT_EXECUTOR_SERVICE,
resultConsumer,
searchRequest,
null,
shardsIter,
timeProvider,
new ClusterState.Builder(new ClusterName("test")).build(),
task,
SearchResponse.Clusters.EMPTY,
null
) {
@Override
protected SearchPhase getNextPhase() {
return new SearchPhase("test") {
@Override
public void run() {
latch.countDown();
}
};
}

@Override
void onShardFailure(int shardIndex, SearchShardTarget shardTarget, Exception e) {
latch.countDown();
fail(e, "Unexpected shard failure");
}
};
action.start();
latch.await();
assertThat(successfulOps.get(), equalTo(numShards));
SearchPhaseController.ReducedQueryPhase phase = action.results.reduce();
assertThat(phase.numReducePhases(), greaterThanOrEqualTo(1));
assertThat(phase.totalHits().value, equalTo(2L));
}
}

}