Skip to content

Commit 1739dab

Browse files
authored
TSDB: fix wrong initial value of tsidOrd in TimeSeriesIndexSearcher (#85713)
Fixes #85711
1 parent a372b54 commit 1739dab

File tree

3 files changed

+107
-26
lines changed

3 files changed

+107
-26
lines changed

docs/changelog/85713.yaml

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,6 @@
1+
pr: 85713
2+
summary: "TSDB: fix wrong initial value of tsidOrd in TimeSeriesIndexSearcher"
3+
area: TSDB
4+
type: bug
5+
issues:
6+
- 85711

server/src/main/java/org/elasticsearch/search/aggregations/timeseries/TimeSeriesIndexSearcher.java

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -208,7 +208,8 @@ int nextDoc() throws IOException {
208208
}
209209

210210
BytesRef getTsid() throws IOException {
211-
scratch.copyBytes(tsids.lookupOrd(tsids.ordValue()));
211+
tsidOrd = tsids.ordValue();
212+
scratch.copyBytes(tsids.lookupOrd(tsidOrd));
212213
return scratch.get();
213214
}
214215

@@ -221,13 +222,11 @@ private boolean isInvalidDoc(int docId) throws IOException {
221222

222223
// true if the TSID ord has changed since the last time we checked
223224
boolean shouldPop() throws IOException {
224-
if (tsidOrd == -1) {
225-
tsidOrd = tsids.ordValue();
226-
} else if (tsidOrd != tsids.ordValue()) {
227-
tsidOrd = tsids.ordValue();
225+
if (tsidOrd != tsids.ordValue()) {
228226
return true;
227+
} else {
228+
return false;
229229
}
230-
return false;
231230
}
232231
}
233232
}

server/src/test/java/org/elasticsearch/search/aggregations/timeseries/TimeSeriesIndexSearcherTests.java

Lines changed: 96 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717
import org.apache.lucene.index.IndexWriterConfig;
1818
import org.apache.lucene.index.NumericDocValues;
1919
import org.apache.lucene.index.SortedDocValues;
20+
import org.apache.lucene.sandbox.search.DocValuesTermsQuery;
2021
import org.apache.lucene.search.IndexSearcher;
2122
import org.apache.lucene.search.MatchAllDocsQuery;
2223
import org.apache.lucene.search.ScoreMode;
@@ -50,26 +51,18 @@ public class TimeSeriesIndexSearcherTests extends ESTestCase {
5051
// Collection should be in order
5152

5253
public void testCollectInOrderAcrossSegments() throws IOException, InterruptedException {
53-
5454
Directory dir = newDirectory();
55-
IndexWriterConfig iwc = newIndexWriterConfig();
56-
boolean tsidReverse = TIME_SERIES_SORT[0].getOrder() == SortOrder.DESC;
57-
boolean timestampReverse = TIME_SERIES_SORT[1].getOrder() == SortOrder.DESC;
58-
Sort sort = new Sort(
59-
new SortField(TimeSeriesIdFieldMapper.NAME, SortField.Type.STRING, tsidReverse),
60-
new SortField(DataStream.TimestampField.FIXED_TIMESTAMP_FIELD, SortField.Type.LONG, timestampReverse)
61-
);
62-
iwc.setIndexSort(sort);
63-
RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
55+
RandomIndexWriter iw = getIndexWriter(dir);
6456

6557
AtomicInteger clock = new AtomicInteger(0);
6658

6759
final int THREADS = 5;
60+
final int DOC_COUNTS = 500;
6861
ExecutorService indexer = Executors.newFixedThreadPool(THREADS);
6962
for (int i = 0; i < THREADS; i++) {
7063
indexer.submit(() -> {
7164
Document doc = new Document();
72-
for (int j = 0; j < 500; j++) {
65+
for (int j = 0; j < DOC_COUNTS; j++) {
7366
String tsid = "tsid" + randomIntBetween(0, 30);
7467
long time = clock.addAndGet(randomIntBetween(0, 10));
7568
doc.clear();
@@ -92,8 +85,98 @@ public void testCollectInOrderAcrossSegments() throws IOException, InterruptedEx
9285

9386
TimeSeriesIndexSearcher indexSearcher = new TimeSeriesIndexSearcher(searcher, List.of());
9487

95-
BucketCollector collector = new BucketCollector() {
88+
BucketCollector collector = getBucketCollector(THREADS * DOC_COUNTS);
89+
90+
indexSearcher.search(new MatchAllDocsQuery(), collector);
91+
collector.postCollection();
92+
93+
reader.close();
94+
dir.close();
95+
}
96+
97+
/**
98+
* this test fixed the wrong init value of tsidOrd
99+
* See https://github.com/elastic/elasticsearch/issues/85711
100+
*/
101+
public void testCollectFromMiddle() throws IOException {
102+
Directory dir = newDirectory();
103+
RandomIndexWriter iw = getIndexWriter(dir);
104+
105+
Document doc = new Document();
106+
final int DOC_COUNTS = 500;
107+
108+
// segment 1
109+
// pre add a value
110+
doc.add(new SortedDocValuesField(TimeSeriesIdFieldMapper.NAME, new BytesRef("tsid")));
111+
doc.add(new NumericDocValuesField(DataStream.TimestampField.FIXED_TIMESTAMP_FIELD, 1));
112+
iw.addDocument(doc);
113+
114+
// segment 1 add value, timestamp is all large then segment 2
115+
for (int j = 0; j < DOC_COUNTS; j++) {
116+
String tsid = "tsid" + randomIntBetween(0, 1);
117+
doc.clear();
118+
doc.add(new SortedDocValuesField(TimeSeriesIdFieldMapper.NAME, new BytesRef(tsid)));
119+
doc.add(new NumericDocValuesField(DataStream.TimestampField.FIXED_TIMESTAMP_FIELD, randomIntBetween(20, 25)));
120+
try {
121+
iw.addDocument(doc);
122+
} catch (IOException e) {
123+
throw new UncheckedIOException(e);
124+
}
125+
}
126+
iw.commit();
127+
128+
// segment 2
129+
// pre add a value
130+
doc.clear();
131+
doc.add(new SortedDocValuesField(TimeSeriesIdFieldMapper.NAME, new BytesRef("tsid")));
132+
doc.add(new NumericDocValuesField(DataStream.TimestampField.FIXED_TIMESTAMP_FIELD, 1));
133+
iw.addDocument(doc);
134+
for (int j = 0; j < DOC_COUNTS; j++) {
135+
String tsid = "tsid" + randomIntBetween(0, 1);
136+
doc.clear();
137+
doc.add(new SortedDocValuesField(TimeSeriesIdFieldMapper.NAME, new BytesRef(tsid)));
138+
doc.add(new NumericDocValuesField(DataStream.TimestampField.FIXED_TIMESTAMP_FIELD, randomIntBetween(10, 15)));
139+
try {
140+
iw.addDocument(doc);
141+
} catch (IOException e) {
142+
throw new UncheckedIOException(e);
143+
}
144+
}
145+
146+
iw.close();
147+
IndexReader reader = DirectoryReader.open(dir);
148+
IndexSearcher searcher = new IndexSearcher(reader);
149+
150+
TimeSeriesIndexSearcher indexSearcher = new TimeSeriesIndexSearcher(searcher, List.of());
151+
152+
BucketCollector collector = getBucketCollector(2 * DOC_COUNTS);
153+
154+
// skip the first doc of segment 1 and 2
155+
indexSearcher.search(new DocValuesTermsQuery("_tsid", List.of(new BytesRef("tsid0"), new BytesRef("tsid1"))), collector);
156+
collector.postCollection();
157+
158+
reader.close();
159+
dir.close();
160+
}
161+
162+
private RandomIndexWriter getIndexWriter(Directory dir) throws IOException {
96163

164+
IndexWriterConfig iwc = newIndexWriterConfig();
165+
boolean tsidReverse = TIME_SERIES_SORT[0].getOrder() == SortOrder.DESC;
166+
boolean timestampReverse = TIME_SERIES_SORT[1].getOrder() == SortOrder.DESC;
167+
Sort sort = new Sort(
168+
new SortField(TimeSeriesIdFieldMapper.NAME, SortField.Type.STRING, tsidReverse),
169+
new SortField(DataStream.TimestampField.FIXED_TIMESTAMP_FIELD, SortField.Type.LONG, timestampReverse)
170+
);
171+
iwc.setIndexSort(sort);
172+
return new RandomIndexWriter(random(), dir, iwc);
173+
}
174+
175+
private BucketCollector getBucketCollector(long totalCount) {
176+
return new BucketCollector() {
177+
178+
boolean tsidReverse = TIME_SERIES_SORT[0].getOrder() == SortOrder.DESC;
179+
boolean timestampReverse = TIME_SERIES_SORT[1].getOrder() == SortOrder.DESC;
97180
BytesRef currentTSID = null;
98181
long currentTimestamp = 0;
99182
long total = 0;
@@ -139,20 +222,13 @@ public void preCollection() throws IOException {
139222

140223
@Override
141224
public void postCollection() throws IOException {
142-
assertEquals(2500, total);
225+
assertEquals(totalCount, total);
143226
}
144227

145228
@Override
146229
public ScoreMode scoreMode() {
147230
return ScoreMode.COMPLETE;
148231
}
149232
};
150-
151-
indexSearcher.search(new MatchAllDocsQuery(), collector);
152-
collector.postCollection();
153-
154-
reader.close();
155-
dir.close();
156-
157233
}
158234
}

0 commit comments

Comments
 (0)