Browse Source

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

Martijn van Groningen 7 months ago
parent
commit
ce3a778fa1

+ 5 - 0
docs/changelog/124477.yaml

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

+ 2 - 1
x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/AbstractDownsampleFieldProducer.java

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

+ 24 - 14
x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/DimensionFieldProducer.java

@@ -7,6 +7,7 @@
 
 package org.elasticsearch.xpack.downsample;
 
+import org.apache.lucene.internal.hppc.IntArrayList;
 import org.elasticsearch.index.fielddata.FormattedDocValues;
 import org.elasticsearch.xcontent.XContentBuilder;
 
@@ -55,13 +56,16 @@ public class DimensionFieldProducer extends AbstractDownsampleFieldProducer {
          * 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.
          */
-        boolean validate(FormattedDocValues docValues, int docId) throws IOException {
-            if (docValues.advanceExact(docId)) {
-                int docValueCount = docValues.docValueCount();
-                for (int i = 0; i < docValueCount; i++) {
-                    var value = docValues.nextValue();
-                    if (value.equals(this.value) == false) {
-                        assert false : "Dimension value changed without tsid change [" + value + "] != [" + this.value + "]";
+        boolean validate(FormattedDocValues docValues, IntArrayList buffer) throws IOException {
+            for (int i = 0; i < buffer.size(); i++) {
+                int docId = buffer.get(i);
+                if (docValues.advanceExact(docId)) {
+                    int docValueCount = docValues.docValueCount();
+                    for (int j = 0; j < docValueCount; j++) {
+                        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
-    public void collect(FormattedDocValues docValues, int docId) throws IOException {
+    public void collect(FormattedDocValues docValues, IntArrayList docIdBuffer) throws IOException {
         if (dimension.isEmpty == false) {
-            assert dimension.validate(docValues, docId);
+            assert dimension.validate(docValues, docIdBuffer);
             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;
         }
-        int docValueCount = docValues.docValueCount();
-        for (int i = 0; i < docValueCount; i++) {
-            this.dimension.collectOnce(docValues.nextValue());
-        }
     }
 
     @Override

+ 140 - 74
x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/DownsampleShardIndexer.java

@@ -10,6 +10,7 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.internal.hppc.IntArrayList;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.MatchNoDocsQuery;
 import org.apache.lucene.search.Query;
@@ -80,6 +81,7 @@ import static org.elasticsearch.core.Strings.format;
 class DownsampleShardIndexer {
 
     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 ByteSizeValue DOWNSAMPLE_BULK_SIZE = ByteSizeValue.of(1, 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 final BulkProcessor2 bulkProcessor;
         private final DownsampleBucketBuilder downsampleBucketBuilder;
+        private final List<LeafDownsampleCollector> leafBucketCollectors = new ArrayList<>();
         private long docsProcessed;
         private long bucketsCreated;
         long lastTimestamp = Long.MAX_VALUE;
@@ -365,83 +368,138 @@ class DownsampleShardIndexer {
                 formattedDocValues[i] = fieldValueFetchers.get(i).getLeaf(ctx);
             }
 
-            return new LeafBucketCollector() {
-                @Override
-                public void collect(int docId, long owningBucketOrd) throws IOException {
-                    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();
-                    if (tsidChanged || timestamp < lastHistoTimestamp) {
-                        lastHistoTimestamp = Math.max(
-                            rounding.round(timestamp),
-                            searchExecutionContext.getIndexSettings().getTimestampBounds().startTime()
-                        );
-                    }
-                    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)
-                        );
-                    }
+            var leafBucketCollector = new LeafDownsampleCollector(aggCtx, docCountProvider, fieldProducers, formattedDocValues);
+            leafBucketCollectors.add(leafBucketCollector);
+            return leafBucketCollector;
+        }
+
+        void bulkCollection() throws IOException {
+            // The leaf bucket collectors with newer timestamp go first, to correctly capture the last value for counters and labels.
+            leafBucketCollectors.sort((o1, o2) -> -Long.compare(o1.firstTimeStampForBulkCollection, o2.firstTimeStampForBulkCollection));
+            for (LeafDownsampleCollector leafBucketCollector : leafBucketCollectors) {
+                leafBucketCollector.leafBulkCollection();
+            }
+        }
 
-                    /*
-                     * 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) {
-                        // 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++;
+        class LeafDownsampleCollector extends LeafBucketCollector {
+
+            final AggregationExecutionContext aggCtx;
+            final DocCountProvider docCountProvider;
+            final FormattedDocValues[] formattedDocValues;
+            final AbstractDownsampleFieldProducer[] fieldProducers;
+
+            // Capture the first timestamp in order to determine which leaf collector's leafBulkCollection() is invoked first.
+            long firstTimeStampForBulkCollection;
+            final IntArrayList docIdBuffer = new IntArrayList(DOCID_BUFFER_SIZE);
+            final long timestampBoundStartTime = searchExecutionContext.getIndexSettings().getTimestampBounds().startTime();
+
+            LeafDownsampleCollector(
+                AggregationExecutionContext aggCtx,
+                DocCountProvider docCountProvider,
+                AbstractDownsampleFieldProducer[] fieldProducers,
+                FormattedDocValues[] formattedDocValues
+            ) {
+                this.aggCtx = aggCtx;
+                this.docCountProvider = docCountProvider;
+                this.fieldProducers = fieldProducers;
+                this.formattedDocValues = formattedDocValues;
+            }
+
+            @Override
+            public void collect(int docId, long owningBucketOrd) throws IOException {
+                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();
+                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);
                     }
 
-                    final int docCount = docCountProvider.getDocCount(docId);
-                    downsampleBucketBuilder.collectDocCount(docCount);
-                    // 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, docId);
+                    // Create new downsample bucket
+                    if (tsidChanged) {
+                        downsampleBucketBuilder.resetTsid(tsidHash, tsidHashOrd, lastHistoTimestamp);
+                    } else {
+                        downsampleBucketBuilder.resetTimestamp(lastHistoTimestamp);
                     }
-                    docsProcessed++;
-                    task.setDocsProcessed(docsProcessed);
+                    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) {
@@ -464,6 +522,7 @@ class DownsampleShardIndexer {
         @Override
         public void postCollection() throws IOException {
             // Flush downsample doc if not empty
+            bulkCollection();
             if (downsampleBucketBuilder.isEmpty() == false) {
                 XContentBuilder doc = downsampleBucketBuilder.buildDownsampleDocument();
                 indexBucket(doc);
@@ -545,8 +604,15 @@ class DownsampleShardIndexer {
             }
         }
 
-        public void collectDocCount(int docCount) {
-            this.docCount += docCount;
+        public void collectDocCount(IntArrayList buffer, DocCountProvider docCountProvider) throws IOException {
+            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 {

+ 21 - 14
x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/LabelFieldProducer.java

@@ -7,6 +7,7 @@
 
 package org.elasticsearch.xpack.downsample;
 
+import org.apache.lucene.internal.hppc.IntArrayList;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.index.fielddata.FormattedDocValues;
 import org.elasticsearch.index.fielddata.HistogramValue;
@@ -114,25 +115,31 @@ abstract class LabelFieldProducer extends AbstractDownsampleFieldProducer {
         }
 
         @Override
-        public void collect(FormattedDocValues docValues, int docId) throws IOException {
+        public void collect(FormattedDocValues docValues, IntArrayList docIdBuffer) throws IOException {
             if (isEmpty() == false) {
                 return;
             }
-            if (docValues.advanceExact(docId) == false) {
-                return;
-            }
 
-            int docValuesCount = docValues.docValueCount();
-            assert docValuesCount > 0;
-            isEmpty = false;
-            if (docValuesCount == 1) {
-                label.collect(docValues.nextValue());
-            } else {
-                Object[] values = new Object[docValuesCount];
-                for (int i = 0; i < docValuesCount; i++) {
-                    values[i] = docValues.nextValue();
+            for (int i = 0; i < docIdBuffer.size(); i++) {
+                int docId = docIdBuffer.get(i);
+                if (docValues.advanceExact(docId) == false) {
+                    continue;
+                }
+                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);
                 }
-                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;
             }
         }
 

+ 14 - 10
x-pack/plugin/downsample/src/main/java/org/elasticsearch/xpack/downsample/MetricFieldProducer.java

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

+ 3 - 2
x-pack/plugin/downsample/src/test/java/org/elasticsearch/xpack/downsample/LabelFieldProducerTests.java

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