mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-04-25 07:37:19 -04:00
Improve downsample performance by buffering docids and do bulk processing. (#124477)
This commit is contained in:
parent
c48f9a9e1c
commit
ce3a778fa1
7 changed files with 207 additions and 113 deletions
5
docs/changelog/124477.yaml
Normal file
5
docs/changelog/124477.yaml
Normal file
|
@ -0,0 +1,5 @@
|
||||||
|
pr: 124477
|
||||||
|
summary: Improve downsample performance by buffering docids and do bulk processing
|
||||||
|
area: Downsampling
|
||||||
|
type: enhancement
|
||||||
|
issues: []
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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());
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue