Improve downsample performance by buffering docids and do bulk processing. (#124477)

This commit is contained in:
Martijn van Groningen 2025-03-13 07:46:08 +01:00 committed by GitHub
parent c48f9a9e1c
commit ce3a778fa1
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
7 changed files with 207 additions and 113 deletions

View file

@ -0,0 +1,5 @@
pr: 124477
summary: Improve downsample performance by buffering docids and do bulk processing
area: Downsampling
type: enhancement
issues: []

View file

@ -7,6 +7,7 @@
package org.elasticsearch.xpack.downsample; package org.elasticsearch.xpack.downsample;
import org.apache.lucene.internal.hppc.IntArrayList;
import org.elasticsearch.index.fielddata.FormattedDocValues; import org.elasticsearch.index.fielddata.FormattedDocValues;
import java.io.IOException; import java.io.IOException;
@ -43,5 +44,5 @@ abstract class AbstractDownsampleFieldProducer implements DownsampleFieldSeriali
return isEmpty; return isEmpty;
} }
public abstract void collect(FormattedDocValues docValues, int docId) throws IOException; public abstract void collect(FormattedDocValues docValues, IntArrayList docIdBuffer) throws IOException;
} }

View file

@ -7,6 +7,7 @@
package org.elasticsearch.xpack.downsample; package org.elasticsearch.xpack.downsample;
import org.apache.lucene.internal.hppc.IntArrayList;
import org.elasticsearch.index.fielddata.FormattedDocValues; import org.elasticsearch.index.fielddata.FormattedDocValues;
import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentBuilder;
@ -55,13 +56,16 @@ public class DimensionFieldProducer extends AbstractDownsampleFieldProducer {
* This is an expensive check, that slows down downsampling significantly. * This is an expensive check, that slows down downsampling significantly.
* Given that index is sorted by tsid as primary key, this shouldn't really happen. * Given that index is sorted by tsid as primary key, this shouldn't really happen.
*/ */
boolean validate(FormattedDocValues docValues, int docId) throws IOException { boolean validate(FormattedDocValues docValues, IntArrayList buffer) throws IOException {
if (docValues.advanceExact(docId)) { for (int i = 0; i < buffer.size(); i++) {
int docValueCount = docValues.docValueCount(); int docId = buffer.get(i);
for (int i = 0; i < docValueCount; i++) { if (docValues.advanceExact(docId)) {
var value = docValues.nextValue(); int docValueCount = docValues.docValueCount();
if (value.equals(this.value) == false) { for (int j = 0; j < docValueCount; j++) {
assert false : "Dimension value changed without tsid change [" + value + "] != [" + this.value + "]"; var value = docValues.nextValue();
if (value.equals(this.value) == false) {
assert false : "Dimension value changed without tsid change [" + value + "] != [" + this.value + "]";
}
} }
} }
} }
@ -81,19 +85,25 @@ public class DimensionFieldProducer extends AbstractDownsampleFieldProducer {
} }
@Override @Override
public void collect(FormattedDocValues docValues, int docId) throws IOException { public void collect(FormattedDocValues docValues, IntArrayList docIdBuffer) throws IOException {
if (dimension.isEmpty == false) { if (dimension.isEmpty == false) {
assert dimension.validate(docValues, docId); assert dimension.validate(docValues, docIdBuffer);
return; return;
} }
if (docValues.advanceExact(docId) == false) { for (int i = 0; i < docIdBuffer.size(); i++) {
int docId = docIdBuffer.get(i);
if (docValues.advanceExact(docId) == false) {
continue;
}
int docValueCount = docValues.docValueCount();
for (int j = 0; j < docValueCount; j++) {
this.dimension.collectOnce(docValues.nextValue());
}
// Only need to record one dimension value from one document, within in the same tsid-and-time-interval bucket values are the
// same.
return; return;
} }
int docValueCount = docValues.docValueCount();
for (int i = 0; i < docValueCount; i++) {
this.dimension.collectOnce(docValues.nextValue());
}
} }
@Override @Override

View file

@ -10,6 +10,7 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.apache.lucene.document.SortedSetDocValuesField; import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.internal.hppc.IntArrayList;
import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
@ -80,6 +81,7 @@ import static org.elasticsearch.core.Strings.format;
class DownsampleShardIndexer { class DownsampleShardIndexer {
private static final Logger logger = LogManager.getLogger(DownsampleShardIndexer.class); private static final Logger logger = LogManager.getLogger(DownsampleShardIndexer.class);
private static final int DOCID_BUFFER_SIZE = 8096;
public static final int DOWNSAMPLE_BULK_ACTIONS = 10000; public static final int DOWNSAMPLE_BULK_ACTIONS = 10000;
public static final ByteSizeValue DOWNSAMPLE_BULK_SIZE = ByteSizeValue.of(1, ByteSizeUnit.MB); public static final ByteSizeValue DOWNSAMPLE_BULK_SIZE = ByteSizeValue.of(1, ByteSizeUnit.MB);
public static final ByteSizeValue DOWNSAMPLE_MAX_BYTES_IN_FLIGHT = ByteSizeValue.of(50, ByteSizeUnit.MB); public static final ByteSizeValue DOWNSAMPLE_MAX_BYTES_IN_FLIGHT = ByteSizeValue.of(50, ByteSizeUnit.MB);
@ -338,6 +340,7 @@ class DownsampleShardIndexer {
private class TimeSeriesBucketCollector extends BucketCollector { private class TimeSeriesBucketCollector extends BucketCollector {
private final BulkProcessor2 bulkProcessor; private final BulkProcessor2 bulkProcessor;
private final DownsampleBucketBuilder downsampleBucketBuilder; private final DownsampleBucketBuilder downsampleBucketBuilder;
private final List<LeafDownsampleCollector> leafBucketCollectors = new ArrayList<>();
private long docsProcessed; private long docsProcessed;
private long bucketsCreated; private long bucketsCreated;
long lastTimestamp = Long.MAX_VALUE; long lastTimestamp = Long.MAX_VALUE;
@ -365,83 +368,138 @@ class DownsampleShardIndexer {
formattedDocValues[i] = fieldValueFetchers.get(i).getLeaf(ctx); formattedDocValues[i] = fieldValueFetchers.get(i).getLeaf(ctx);
} }
return new LeafBucketCollector() { var leafBucketCollector = new LeafDownsampleCollector(aggCtx, docCountProvider, fieldProducers, formattedDocValues);
@Override leafBucketCollectors.add(leafBucketCollector);
public void collect(int docId, long owningBucketOrd) throws IOException { return leafBucketCollector;
task.addNumReceived(1); }
final BytesRef tsidHash = aggCtx.getTsidHash();
assert tsidHash != null : "Document without [" + TimeSeriesIdFieldMapper.NAME + "] field was found.";
final int tsidHashOrd = aggCtx.getTsidHashOrd();
final long timestamp = timestampField.resolution().roundDownToMillis(aggCtx.getTimestamp());
boolean tsidChanged = tsidHashOrd != downsampleBucketBuilder.tsidOrd(); void bulkCollection() throws IOException {
if (tsidChanged || timestamp < lastHistoTimestamp) { // The leaf bucket collectors with newer timestamp go first, to correctly capture the last value for counters and labels.
lastHistoTimestamp = Math.max( leafBucketCollectors.sort((o1, o2) -> -Long.compare(o1.firstTimeStampForBulkCollection, o2.firstTimeStampForBulkCollection));
rounding.round(timestamp), for (LeafDownsampleCollector leafBucketCollector : leafBucketCollectors) {
searchExecutionContext.getIndexSettings().getTimestampBounds().startTime() leafBucketCollector.leafBulkCollection();
); }
} }
task.setLastSourceTimestamp(timestamp);
task.setLastTargetTimestamp(lastHistoTimestamp);
if (logger.isTraceEnabled()) { class LeafDownsampleCollector extends LeafBucketCollector {
logger.trace(
"Doc: [{}] - _tsid: [{}], @timestamp: [{}] -> downsample bucket ts: [{}]",
docId,
DocValueFormat.TIME_SERIES_ID.format(tsidHash),
timestampFormat.format(timestamp),
timestampFormat.format(lastHistoTimestamp)
);
}
/* final AggregationExecutionContext aggCtx;
* Sanity checks to ensure that we receive documents in the correct order final DocCountProvider docCountProvider;
* - _tsid must be sorted in ascending order final FormattedDocValues[] formattedDocValues;
* - @timestamp must be sorted in descending order within the same _tsid final AbstractDownsampleFieldProducer[] fieldProducers;
*/
BytesRef lastTsid = downsampleBucketBuilder.tsid();
assert lastTsid == null || lastTsid.compareTo(tsidHash) <= 0
: "_tsid is not sorted in ascending order: ["
+ DocValueFormat.TIME_SERIES_ID.format(lastTsid)
+ "] -> ["
+ DocValueFormat.TIME_SERIES_ID.format(tsidHash)
+ "]";
assert tsidHash.equals(lastTsid) == false || lastTimestamp >= timestamp
: "@timestamp is not sorted in descending order: ["
+ timestampFormat.format(lastTimestamp)
+ "] -> ["
+ timestampFormat.format(timestamp)
+ "]";
lastTimestamp = timestamp;
if (tsidChanged || downsampleBucketBuilder.timestamp() != lastHistoTimestamp) { // Capture the first timestamp in order to determine which leaf collector's leafBulkCollection() is invoked first.
// Flush downsample doc if not empty long firstTimeStampForBulkCollection;
if (downsampleBucketBuilder.isEmpty() == false) { final IntArrayList docIdBuffer = new IntArrayList(DOCID_BUFFER_SIZE);
XContentBuilder doc = downsampleBucketBuilder.buildDownsampleDocument(); final long timestampBoundStartTime = searchExecutionContext.getIndexSettings().getTimestampBounds().startTime();
indexBucket(doc);
}
// Create new downsample bucket LeafDownsampleCollector(
if (tsidChanged) { AggregationExecutionContext aggCtx,
downsampleBucketBuilder.resetTsid(tsidHash, tsidHashOrd, lastHistoTimestamp); DocCountProvider docCountProvider,
} else { AbstractDownsampleFieldProducer[] fieldProducers,
downsampleBucketBuilder.resetTimestamp(lastHistoTimestamp); FormattedDocValues[] formattedDocValues
} ) {
bucketsCreated++; this.aggCtx = aggCtx;
} this.docCountProvider = docCountProvider;
this.fieldProducers = fieldProducers;
this.formattedDocValues = formattedDocValues;
}
final int docCount = docCountProvider.getDocCount(docId); @Override
downsampleBucketBuilder.collectDocCount(docCount); public void collect(int docId, long owningBucketOrd) throws IOException {
// Iterate over all field values and collect the doc_values for this docId task.addNumReceived(1);
for (int i = 0; i < fieldProducers.length; i++) { final BytesRef tsidHash = aggCtx.getTsidHash();
AbstractDownsampleFieldProducer fieldProducer = fieldProducers[i]; assert tsidHash != null : "Document without [" + TimeSeriesIdFieldMapper.NAME + "] field was found.";
FormattedDocValues docValues = formattedDocValues[i]; final int tsidHashOrd = aggCtx.getTsidHashOrd();
fieldProducer.collect(docValues, docId); final long timestamp = timestampField.resolution().roundDownToMillis(aggCtx.getTimestamp());
}
docsProcessed++; boolean tsidChanged = tsidHashOrd != downsampleBucketBuilder.tsidOrd();
task.setDocsProcessed(docsProcessed); if (tsidChanged || timestamp < lastHistoTimestamp) {
lastHistoTimestamp = Math.max(rounding.round(timestamp), timestampBoundStartTime);
} }
}; task.setLastSourceTimestamp(timestamp);
task.setLastTargetTimestamp(lastHistoTimestamp);
if (logger.isTraceEnabled()) {
logger.trace(
"Doc: [{}] - _tsid: [{}], @timestamp: [{}] -> downsample bucket ts: [{}]",
docId,
DocValueFormat.TIME_SERIES_ID.format(tsidHash),
timestampFormat.format(timestamp),
timestampFormat.format(lastHistoTimestamp)
);
}
/*
* Sanity checks to ensure that we receive documents in the correct order
* - _tsid must be sorted in ascending order
* - @timestamp must be sorted in descending order within the same _tsid
*/
BytesRef lastTsid = downsampleBucketBuilder.tsid();
assert lastTsid == null || lastTsid.compareTo(tsidHash) <= 0
: "_tsid is not sorted in ascending order: ["
+ DocValueFormat.TIME_SERIES_ID.format(lastTsid)
+ "] -> ["
+ DocValueFormat.TIME_SERIES_ID.format(tsidHash)
+ "]";
assert tsidHash.equals(lastTsid) == false || lastTimestamp >= timestamp
: "@timestamp is not sorted in descending order: ["
+ timestampFormat.format(lastTimestamp)
+ "] -> ["
+ timestampFormat.format(timestamp)
+ "]";
lastTimestamp = timestamp;
if (tsidChanged || downsampleBucketBuilder.timestamp() != lastHistoTimestamp) {
bulkCollection();
// Flush downsample doc if not empty
if (downsampleBucketBuilder.isEmpty() == false) {
XContentBuilder doc = downsampleBucketBuilder.buildDownsampleDocument();
indexBucket(doc);
}
// Create new downsample bucket
if (tsidChanged) {
downsampleBucketBuilder.resetTsid(tsidHash, tsidHashOrd, lastHistoTimestamp);
} else {
downsampleBucketBuilder.resetTimestamp(lastHistoTimestamp);
}
bucketsCreated++;
}
if (docIdBuffer.isEmpty()) {
firstTimeStampForBulkCollection = aggCtx.getTimestamp();
}
// buffer.add() always delegates to system.arraycopy() and checks buffer size for resizing purposes:
docIdBuffer.buffer[docIdBuffer.elementsCount++] = docId;
if (docIdBuffer.size() == DOCID_BUFFER_SIZE) {
bulkCollection();
}
}
void leafBulkCollection() throws IOException {
if (docIdBuffer.isEmpty()) {
return;
}
if (logger.isDebugEnabled()) {
logger.debug("buffered {} docids", docIdBuffer.size());
}
downsampleBucketBuilder.collectDocCount(docIdBuffer, docCountProvider);
// Iterate over all field values and collect the doc_values for this docId
for (int i = 0; i < fieldProducers.length; i++) {
AbstractDownsampleFieldProducer fieldProducer = fieldProducers[i];
FormattedDocValues docValues = formattedDocValues[i];
fieldProducer.collect(docValues, docIdBuffer);
}
docsProcessed += docIdBuffer.size();
task.setDocsProcessed(docsProcessed);
// buffer.clean() also overwrites all slots with zeros
docIdBuffer.elementsCount = 0;
}
} }
private void indexBucket(XContentBuilder doc) { private void indexBucket(XContentBuilder doc) {
@ -464,6 +522,7 @@ class DownsampleShardIndexer {
@Override @Override
public void postCollection() throws IOException { public void postCollection() throws IOException {
// Flush downsample doc if not empty // Flush downsample doc if not empty
bulkCollection();
if (downsampleBucketBuilder.isEmpty() == false) { if (downsampleBucketBuilder.isEmpty() == false) {
XContentBuilder doc = downsampleBucketBuilder.buildDownsampleDocument(); XContentBuilder doc = downsampleBucketBuilder.buildDownsampleDocument();
indexBucket(doc); indexBucket(doc);
@ -545,8 +604,15 @@ class DownsampleShardIndexer {
} }
} }
public void collectDocCount(int docCount) { public void collectDocCount(IntArrayList buffer, DocCountProvider docCountProvider) throws IOException {
this.docCount += docCount; if (docCountProvider.alwaysOne()) {
this.docCount += buffer.size();
} else {
for (int i = 0; i < buffer.size(); i++) {
int docId = buffer.get(i);
this.docCount += docCountProvider.getDocCount(docId);
}
}
} }
public XContentBuilder buildDownsampleDocument() throws IOException { public XContentBuilder buildDownsampleDocument() throws IOException {

View file

@ -7,6 +7,7 @@
package org.elasticsearch.xpack.downsample; package org.elasticsearch.xpack.downsample;
import org.apache.lucene.internal.hppc.IntArrayList;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.fielddata.FormattedDocValues; import org.elasticsearch.index.fielddata.FormattedDocValues;
import org.elasticsearch.index.fielddata.HistogramValue; import org.elasticsearch.index.fielddata.HistogramValue;
@ -114,25 +115,31 @@ abstract class LabelFieldProducer extends AbstractDownsampleFieldProducer {
} }
@Override @Override
public void collect(FormattedDocValues docValues, int docId) throws IOException { public void collect(FormattedDocValues docValues, IntArrayList docIdBuffer) throws IOException {
if (isEmpty() == false) { if (isEmpty() == false) {
return; return;
} }
if (docValues.advanceExact(docId) == false) {
return;
}
int docValuesCount = docValues.docValueCount(); for (int i = 0; i < docIdBuffer.size(); i++) {
assert docValuesCount > 0; int docId = docIdBuffer.get(i);
isEmpty = false; if (docValues.advanceExact(docId) == false) {
if (docValuesCount == 1) { continue;
label.collect(docValues.nextValue());
} else {
Object[] values = new Object[docValuesCount];
for (int i = 0; i < docValuesCount; i++) {
values[i] = docValues.nextValue();
} }
label.collect(values); int docValuesCount = docValues.docValueCount();
assert docValuesCount > 0;
isEmpty = false;
if (docValuesCount == 1) {
label.collect(docValues.nextValue());
} else {
var values = new Object[docValuesCount];
for (int j = 0; j < docValuesCount; j++) {
values[j] = docValues.nextValue();
}
label.collect(values);
}
// Only need to record one label value from one document, within in the same tsid-and-time-interval we only keep the first
// with downsampling.
return;
} }
} }

View file

@ -7,6 +7,7 @@
package org.elasticsearch.xpack.downsample; package org.elasticsearch.xpack.downsample;
import org.apache.lucene.internal.hppc.IntArrayList;
import org.elasticsearch.index.fielddata.FormattedDocValues; import org.elasticsearch.index.fielddata.FormattedDocValues;
import org.elasticsearch.search.aggregations.metrics.CompensatedSum; import org.elasticsearch.search.aggregations.metrics.CompensatedSum;
import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentBuilder;
@ -53,14 +54,17 @@ abstract sealed class MetricFieldProducer extends AbstractDownsampleFieldProduce
} }
@Override @Override
public void collect(FormattedDocValues docValues, int docId) throws IOException { public void collect(FormattedDocValues docValues, IntArrayList docIdBuffer) throws IOException {
if (docValues.advanceExact(docId) == false) { for (int i = 0; i < docIdBuffer.size(); i++) {
return; int docId = docIdBuffer.get(i);
} if (docValues.advanceExact(docId) == false) {
int docValuesCount = docValues.docValueCount(); continue;
for (int i = 0; i < docValuesCount; i++) { }
Number num = (Number) docValues.nextValue(); int docValuesCount = docValues.docValueCount();
collect(num); for (int j = 0; j < docValuesCount; j++) {
Number num = (Number) docValues.nextValue();
collect(num);
}
} }
} }
@ -236,13 +240,13 @@ abstract sealed class MetricFieldProducer extends AbstractDownsampleFieldProduce
} }
@Override @Override
public void collect(FormattedDocValues docValues, int docId) throws IOException { public void collect(FormattedDocValues docValues, IntArrayList docIdBuffer) throws IOException {
// Counter producers only collect the last_value. Since documents are // Counter producers only collect the last_value. Since documents are
// collected by descending timestamp order, the producer should only // collected by descending timestamp order, the producer should only
// process the first value for every tsid. So, it will only collect the // process the first value for every tsid. So, it will only collect the
// field if no value has been set before. // field if no value has been set before.
if (isEmpty()) { if (isEmpty()) {
super.collect(docValues, docId); super.collect(docValues, docIdBuffer);
} }
} }

View file

@ -7,6 +7,7 @@
package org.elasticsearch.xpack.downsample; package org.elasticsearch.xpack.downsample;
import org.apache.lucene.internal.hppc.IntArrayList;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.index.fielddata.FormattedDocValues; import org.elasticsearch.index.fielddata.FormattedDocValues;
import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.AggregatorTestCase;
@ -93,7 +94,7 @@ public class LabelFieldProducerTests extends AggregatorTestCase {
return "aaaa"; return "aaaa";
} }
}; };
producer.collect(docValues, 1); producer.collect(docValues, IntArrayList.from(1));
// producer.collect("dummy", "aaaa"); // producer.collect("dummy", "aaaa");
assertFalse(producer.isEmpty()); assertFalse(producer.isEmpty());
assertEquals("aaaa", producer.label().get()); assertEquals("aaaa", producer.label().get());
@ -129,7 +130,7 @@ public class LabelFieldProducerTests extends AggregatorTestCase {
} }
}; };
producer.collect(docValues, 1); producer.collect(docValues, IntArrayList.from(1));
assertFalse(producer.isEmpty()); assertFalse(producer.isEmpty());
assertEquals("a\0value_a", (((Object[]) producer.label().get())[0]).toString()); assertEquals("a\0value_a", (((Object[]) producer.label().get())[0]).toString());
assertEquals("b\0value_b", (((Object[]) producer.label().get())[1]).toString()); assertEquals("b\0value_b", (((Object[]) producer.label().get())[1]).toString());