Test bug in TimeSeriesIndexSearcherTests (#83730)

This test checks that documents are presented in tsid and then timestamp
order to collectors; it was not taking a deep copy of the tsid and so when
a change in tsid stayed on the same segment we could get inaccurate
comparisons as the BytesRef would change out from underneath us and
would compare as equal to the old timestamp, meaning that the check for
strictly increasing timestamp would fail.

Fixes #83647
This commit is contained in:
Alan Woodward 2022-02-10 09:24:57 +00:00 committed by GitHub
parent 9d867cd5ca
commit fed07a54ba
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
2 changed files with 11 additions and 6 deletions

View file

@ -20,6 +20,7 @@ import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Weight; import org.apache.lucene.search.Weight;
import org.apache.lucene.util.Bits; import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.PriorityQueue; import org.apache.lucene.util.PriorityQueue;
import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper;
@ -99,7 +100,7 @@ public class TimeSeriesIndexSearcher {
it.remove(); it.remove();
continue; continue;
} }
BytesRef tsid = leafWalker.tsids.lookupOrd(leafWalker.tsids.ordValue()); BytesRef tsid = leafWalker.getTsid();
if (currentTsid == null) { if (currentTsid == null) {
currentTsid = tsid; currentTsid = tsid;
} }
@ -136,6 +137,7 @@ public class TimeSeriesIndexSearcher {
private final DocIdSetIterator iterator; private final DocIdSetIterator iterator;
private final SortedDocValues tsids; private final SortedDocValues tsids;
private final SortedNumericDocValues timestamps; // TODO can we have this just a NumericDocValues? private final SortedNumericDocValues timestamps; // TODO can we have this just a NumericDocValues?
private final BytesRefBuilder scratch = new BytesRefBuilder();
int docId = -1; int docId = -1;
int tsidOrd; int tsidOrd;
long timestamp; long timestamp;
@ -168,6 +170,11 @@ public class TimeSeriesIndexSearcher {
return docId; return docId;
} }
BytesRef getTsid() throws IOException {
scratch.copyBytes(tsids.lookupOrd(tsids.ordValue()));
return scratch.get();
}
// invalid if the doc is deleted or if it doesn't have a tsid or timestamp entry // invalid if the doc is deleted or if it doesn't have a tsid or timestamp entry
private boolean isInvalidDoc(int docId) throws IOException { private boolean isInvalidDoc(int docId) throws IOException {
return (liveDocs != null && liveDocs.get(docId) == false) return (liveDocs != null && liveDocs.get(docId) == false)

View file

@ -26,7 +26,6 @@ import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField; import org.apache.lucene.search.SortField;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase.AwaitsFix;
import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.DataStream;
import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper; import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper;
import org.elasticsearch.search.aggregations.BucketCollector; import org.elasticsearch.search.aggregations.BucketCollector;
@ -40,7 +39,6 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/83647")
public class TimeSeriesIndexSearcherTests extends ESTestCase { public class TimeSeriesIndexSearcherTests extends ESTestCase {
// Index a random set of docs with timestamp and tsid with the tsid/timestamp sort order // Index a random set of docs with timestamp and tsid with the tsid/timestamp sort order
@ -108,13 +106,13 @@ public class TimeSeriesIndexSearcherTests extends ESTestCase {
BytesRef latestTSID = tsid.lookupOrd(tsid.ordValue()); BytesRef latestTSID = tsid.lookupOrd(tsid.ordValue());
long latestTimestamp = timestamp.longValue(); long latestTimestamp = timestamp.longValue();
if (currentTSID != null) { if (currentTSID != null) {
assertTrue(latestTSID.compareTo(currentTSID) >= 0); assertTrue(currentTSID + "->" + latestTSID.utf8ToString(), latestTSID.compareTo(currentTSID) >= 0);
if (latestTSID.equals(currentTSID)) { if (latestTSID.equals(currentTSID)) {
assertTrue(latestTimestamp >= currentTimestamp); assertTrue(currentTimestamp + "->" + latestTimestamp, latestTimestamp >= currentTimestamp);
} }
} }
currentTimestamp = latestTimestamp; currentTimestamp = latestTimestamp;
currentTSID = latestTSID; currentTSID = BytesRef.deepCopyOf(latestTSID);
total++; total++;
} }
}; };