Skip to content

Commit

Permalink
TSDB: Change _tsid field to SortedDocValuesField (#83045)
Browse files Browse the repository at this point in the history
Since _tsid cannot be a multi-value field, this PR modifies the TimeSeriesIdFieldMapper 
so that _tsid is added as a SortedDocValuesField (instead of a SortedSetDocValuesField)

Relates to #80276
  • Loading branch information
csoulios authored Jan 26, 2022
1 parent 7f321fb commit 40cab17
Show file tree
Hide file tree
Showing 5 changed files with 119 additions and 15 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
/*
* 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 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/

package org.elasticsearch.index.fielddata.plain;

import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.OrdinalMap;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.SortedSetSortField;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.LeafOrdinalsFieldData;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.script.field.ToScriptField;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import org.elasticsearch.search.sort.BucketedSort;
import org.elasticsearch.search.sort.SortOrder;

import static org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.sortMissingLast;

public class SortedOrdinalsIndexFieldData extends AbstractIndexOrdinalsFieldData {

public static class Builder implements IndexFieldData.Builder {
private final String name;
private final ToScriptField<SortedSetDocValues> toScriptField;
private final ValuesSourceType valuesSourceType;

public Builder(String name, ValuesSourceType valuesSourceType, ToScriptField<SortedSetDocValues> toScriptField) {
this.name = name;
this.toScriptField = toScriptField;
this.valuesSourceType = valuesSourceType;
}

@Override
public SortedOrdinalsIndexFieldData build(IndexFieldDataCache cache, CircuitBreakerService breakerService) {
return new SortedOrdinalsIndexFieldData(cache, name, valuesSourceType, breakerService, toScriptField);
}
}

public SortedOrdinalsIndexFieldData(
IndexFieldDataCache cache,
String fieldName,
ValuesSourceType valuesSourceType,
CircuitBreakerService breakerService,
ToScriptField<SortedSetDocValues> toScriptField
) {
super(fieldName, valuesSourceType, cache, breakerService, toScriptField);
}

@Override
public SortField sortField(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested, boolean reverse) {
SortField sortField = new SortField(getFieldName(), SortField.Type.STRING, reverse);
sortField.setMissingValue(
sortMissingLast(missingValue) ^ reverse ? SortedSetSortField.STRING_LAST : SortedSetSortField.STRING_FIRST
);
return sortField;
}

@Override
public BucketedSort newBucketedSort(
BigArrays bigArrays,
Object missingValue,
MultiValueMode sortMode,
Nested nested,
SortOrder sortOrder,
DocValueFormat format,
int bucketSize,
BucketedSort.ExtraData extra
) {
throw new IllegalArgumentException("only supported on numeric fields");
}

@Override
public LeafOrdinalsFieldData load(LeafReaderContext context) {
// Doc value fields are loaded using Lucene's DocValues#getSortedSet
// that can happily load SortedDocValues as well.
return new SortedSetBytesLeafFieldData(context.reader(), getFieldName(), toScriptField);
}

@Override
public LeafOrdinalsFieldData loadDirect(LeafReaderContext context) {
return load(context);
}

@Override
public OrdinalMap getOrdinalMap() {
return null;
}

@Override
public boolean supportsGlobalOrdinalsMapping() {
return true;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@

package org.elasticsearch.index.mapper;

import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
Expand All @@ -21,7 +21,7 @@
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.index.fielddata.plain.SortedSetOrdinalsIndexFieldData;
import org.elasticsearch.index.fielddata.plain.SortedOrdinalsIndexFieldData;
import org.elasticsearch.index.query.SearchExecutionContext;
import org.elasticsearch.script.field.DelegateDocValuesField;
import org.elasticsearch.search.DocValueFormat;
Expand Down Expand Up @@ -116,7 +116,7 @@ public DocValueFormat docValueFormat(String format, ZoneId timeZone) {
public IndexFieldData.Builder fielddataBuilder(String fullyQualifiedIndexName, Supplier<SearchLookup> searchLookup) {
failIfNoDocValues();
// TODO don't leak the TSID's binary format into the script
return new SortedSetOrdinalsIndexFieldData.Builder(
return new SortedOrdinalsIndexFieldData.Builder(
name(),
CoreValuesSourceType.KEYWORD,
(dv, n) -> new DelegateDocValuesField(
Expand All @@ -143,7 +143,7 @@ public void postParse(DocumentParserContext context) throws IOException {
// SortedMap is expected to be sorted by key (field name)
SortedMap<String, BytesReference> dimensionFields = context.doc().getDimensionBytes();
BytesReference timeSeriesId = buildTsidField(dimensionFields);
context.doc().add(new SortedSetDocValuesField(fieldType().name(), timeSeriesId.toBytesRef()));
context.doc().add(new SortedDocValuesField(fieldType().name(), timeSeriesId.toBytesRef()));
}

public static BytesReference buildTsidField(SortedMap<String, BytesReference> dimensionFields) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,9 +8,10 @@

package org.elasticsearch.search.aggregations.timeseries;

import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LeafCollector;
Expand Down Expand Up @@ -80,7 +81,7 @@ private static class LeafWalker {
private final LeafCollector collector;
private final Bits liveDocs;
private final DocIdSetIterator iterator;
private final SortedSetDocValues tsids;
private final SortedDocValues tsids;
private final SortedNumericDocValues timestamps;
final int docBase;
int docId;
Expand All @@ -93,8 +94,8 @@ private static class LeafWalker {
this.collector.setScorer(scorer);
iterator = scorer.iterator();
docBase = context.docBase;
tsids = context.reader().getSortedSetDocValues(TimeSeriesIdFieldMapper.NAME);
timestamps = context.reader().getSortedNumericDocValues(DataStream.TimestampField.FIXED_TIMESTAMP_FIELD);
tsids = DocValues.getSorted(context.reader(), TimeSeriesIdFieldMapper.NAME);
timestamps = DocValues.getSortedNumeric(context.reader(), DataStream.TimestampField.FIXED_TIMESTAMP_FIELD);
}

void collectCurrent() throws IOException {
Expand All @@ -106,7 +107,7 @@ boolean next() throws IOException {
docId = iterator.nextDoc();
if (docId != DocIdSetIterator.NO_MORE_DOCS && (liveDocs == null || liveDocs.get(docId))) {
if (tsids.advanceExact(docId)) {
BytesRef tsid = tsids.lookupOrd(tsids.nextOrd());
BytesRef tsid = tsids.lookupOrd(tsids.ordValue());
if (timestamps.advanceExact(docId)) {
this.timestamp = timestamps.nextValue();
if (tsid.equals(this.tsid) == false) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,8 @@
import org.apache.lucene.document.DoubleDocValuesField;
import org.apache.lucene.document.FloatDocValuesField;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.MatchAllDocsQuery;
Expand Down Expand Up @@ -102,7 +102,7 @@ public static void writeTS(RandomIndexWriter iw, long timestamp, Object[] dimens
try (BytesStreamOutput out = new BytesStreamOutput()) {
TimeSeriesIdFieldMapper.encodeTsid(out, dimensionFields);
BytesReference timeSeriesId = out.bytes();
fields.add(new SortedSetDocValuesField(TimeSeriesIdFieldMapper.NAME, timeSeriesId.toBytesRef()));
fields.add(new SortedDocValuesField(TimeSeriesIdFieldMapper.NAME, timeSeriesId.toBytesRef()));
}
// TODO: Handle metrics
iw.addDocument(fields.stream().toList());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,6 @@
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.SortedNumericSortField;
import org.apache.lucene.search.SortedSetSelector;
import org.apache.lucene.search.SortedSetSortField;
import org.apache.lucene.search.Weight;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Accountable;
Expand Down Expand Up @@ -664,8 +662,8 @@ protected <T extends AggregationBuilder, V extends InternalAggregation> void tes
IndexWriterConfig config = LuceneTestCase.newIndexWriterConfig(random(), new MockAnalyzer(random()));
if (timeSeries) {
Sort sort = new Sort(
new SortedSetSortField(TimeSeriesIdFieldMapper.NAME, false, SortedSetSelector.Type.MAX),
new SortedNumericSortField(DataStreamTimestampFieldMapper.DEFAULT_PATH, SortField.Type.LONG)
new SortField(TimeSeriesIdFieldMapper.NAME, SortField.Type.STRING, false),
new SortedNumericSortField(DataStreamTimestampFieldMapper.DEFAULT_PATH, SortField.Type.LONG, true)
);
config.setIndexSort(sort);
}
Expand Down

0 comments on commit 40cab17

Please sign in to comment.