Browse Source

Add support for dynamic pruning to cardinality aggregations on low-cardinality keyword fields. (#92060)

On low-cardinality keyword fields, the `cardinality` aggregation currently uses
the `global_ordinals` execution mode most of the time, which consists of
collecting all documents that match the query, reading ordinals of the values
that these documents contain, and setting bits in a bitset for these ordinals.

This commit introduces a feedback loop between the query and the `cardinality`
aggregator, which allows the query to skip documents that only contain values
that have already been seen by the `cardinality` aggregator. On the `nyc_taxis`
dataset, a `match_all` query and the `vendor_id` field (2 unique values), the
`cardinality` aggregation went from 3s to 3ms. The speedup would certainly not
be as good in all cases, but I would still expect in to be very significant in
many cases.
Adrien Grand 2 years ago
parent
commit
79ad42cbd2

+ 6 - 0
docs/changelog/92060.yaml

@@ -0,0 +1,6 @@
+pr: 92060
+summary: Add support for dynamic pruning to cardinality aggregations on low-cardinality
+  keyword fields
+area: Aggregations
+type: enhancement
+issues: []

+ 12 - 0
server/src/main/java/org/elasticsearch/common/util/BitArray.java

@@ -39,6 +39,18 @@ public final class BitArray implements Releasable {
         bits.set(wordNum, bits.get(wordNum) | bitmask(index));
     }
 
+    /**
+     * Set the {@code index}th bit and return {@code true} if the bit was set already.
+     */
+    public boolean getAndSet(long index) {
+        long wordNum = wordNum(index);
+        bits = bigArrays.grow(bits, wordNum + 1);
+        long word = bits.get(wordNum);
+        long bitMask = bitmask(index);
+        bits.set(wordNum, word | bitMask);
+        return (word & bitMask) != 0;
+    }
+
     /** this = this OR other */
     public void or(BitArray other) {
         or(other.bits);

+ 7 - 0
server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorFactory.java

@@ -157,10 +157,17 @@ public class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory
                 if (valuesSourceConfig.hasValues()) {
                     if (valuesSourceConfig.getValuesSource() instanceof final ValuesSource.Bytes.WithOrdinals source) {
                         if (executionMode.useGlobalOrdinals(context, source, precision)) {
+                            final String field;
+                            if (valuesSourceConfig.alignesWithSearchIndex()) {
+                                field = valuesSourceConfig.fieldType().name();
+                            } else {
+                                field = null;
+                            }
                             final long maxOrd = source.globalMaxOrd(context.searcher().getIndexReader());
                             return new GlobalOrdCardinalityAggregator(
                                 name,
                                 source,
+                                field,
                                 precision,
                                 Math.toIntExact(maxOrd),
                                 context,

+ 206 - 1
server/src/main/java/org/elasticsearch/search/aggregations/metrics/GlobalOrdCardinalityAggregator.java

@@ -8,9 +8,16 @@
 
 package org.elasticsearch.search.aggregations.metrics;
 
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.PriorityQueue;
 import org.elasticsearch.common.hash.MurmurHash3;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.BitArray;
@@ -26,7 +33,10 @@ import org.elasticsearch.search.aggregations.support.AggregationContext;
 import org.elasticsearch.search.aggregations.support.ValuesSource;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
+import java.util.function.BiConsumer;
 
 /**
  * An aggregator that computes approximate counts of unique values
@@ -34,10 +44,24 @@ import java.util.Map;
  */
 public class GlobalOrdCardinalityAggregator extends NumericMetricsAggregator.SingleValue {
 
+    // Don't try to dynamically prune fields that have more than 1024 unique terms, there is a chance we never get to 128 unseen terms, and
+    // we'd be paying the overhead of dynamic pruning without getting any benefits.
+    private static final int MAX_FIELD_CARDINALITY_FOR_DYNAMIC_PRUNING = 1024;
+
+    // Only start dynamic pruning when 128 ordinals or less have not been seen yet.
+    private static final int MAX_TERMS_FOR_DYNAMIC_PRUNING = 128;
+
     private final ValuesSource.Bytes.WithOrdinals valuesSource;
+    // The field that this cardinality aggregation runs on, or null if there is no field, or the field doesn't directly map to an index
+    // field.
+    private final String field;
     private final BigArrays bigArrays;
     private final int maxOrd;
     private final int precision;
+    private int dynamicPruningAttempts;
+    private int dynamicPruningSuccess;
+    private int bruteForce;
+    private int noData;
 
     // Build at post-collection phase
     @Nullable
@@ -48,6 +72,7 @@ public class GlobalOrdCardinalityAggregator extends NumericMetricsAggregator.Sin
     public GlobalOrdCardinalityAggregator(
         String name,
         ValuesSource.Bytes.WithOrdinals valuesSource,
+        String field,
         int precision,
         int maxOrd,
         AggregationContext context,
@@ -56,6 +81,7 @@ public class GlobalOrdCardinalityAggregator extends NumericMetricsAggregator.Sin
     ) throws IOException {
         super(name, context, parent, metadata);
         this.valuesSource = valuesSource;
+        this.field = field;
         this.precision = precision;
         this.maxOrd = maxOrd;
         this.bigArrays = context.bigArrays();
@@ -64,12 +90,182 @@ public class GlobalOrdCardinalityAggregator extends NumericMetricsAggregator.Sin
 
     @Override
     public ScoreMode scoreMode() {
-        return valuesSource.needsScores() ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES;
+        if (field != null && valuesSource.needsScores() == false && maxOrd <= MAX_FIELD_CARDINALITY_FOR_DYNAMIC_PRUNING) {
+            return ScoreMode.TOP_DOCS;
+        } else if (valuesSource.needsScores()) {
+            return ScoreMode.COMPLETE;
+        } else {
+            return ScoreMode.COMPLETE_NO_SCORES;
+        }
+    }
+
+    /**
+     * A competitive iterator that helps only collect values that have not been collected so far.
+     */
+    private class CompetitiveIterator extends DocIdSetIterator {
+
+        private final BitArray visitedOrds;
+        private long numNonVisitedOrds;
+        private final TermsEnum indexTerms;
+        private final DocIdSetIterator docsWithField;
+
+        CompetitiveIterator(int numNonVisitedOrds, BitArray visitedOrds, Terms indexTerms, DocIdSetIterator docsWithField)
+            throws IOException {
+            this.visitedOrds = visitedOrds;
+            this.numNonVisitedOrds = numNonVisitedOrds;
+            this.indexTerms = Objects.requireNonNull(indexTerms).iterator();
+            this.docsWithField = docsWithField;
+        }
+
+        private Map<Long, PostingsEnum> nonVisitedOrds;
+        private PriorityQueue<PostingsEnum> nonVisitedPostings;
+
+        private int doc = -1;
+
+        @Override
+        public int docID() {
+            return doc;
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+            return advance(doc + 1);
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+            if (nonVisitedPostings == null) {
+                // We haven't started pruning yet, iterate on docs that have a value. This may already help a lot on sparse fields.
+                return doc = docsWithField.advance(target);
+            } else if (nonVisitedPostings.size() == 0) {
+                return doc = DocIdSetIterator.NO_MORE_DOCS;
+            } else {
+                PostingsEnum top = nonVisitedPostings.top();
+                while (top.docID() < target) {
+                    top.advance(target);
+                    top = nonVisitedPostings.updateTop();
+                }
+                return doc = top.docID();
+            }
+        }
+
+        @Override
+        public long cost() {
+            return docsWithField.cost();
+        }
+
+        void startPruning() throws IOException {
+            dynamicPruningSuccess++;
+            nonVisitedOrds = new HashMap<>();
+            // TODO: iterate the bitset using a `nextClearBit` operation?
+            for (long ord = 0; ord < maxOrd; ++ord) {
+                if (visitedOrds.get(ord)) {
+                    continue;
+                }
+                BytesRef term = values.lookupOrd(ord);
+                if (indexTerms.seekExact(term) == false) {
+                    // This global ordinal maps to a value that doesn't exist in this segment
+                    continue;
+                }
+                nonVisitedOrds.put(ord, indexTerms.postings(null, PostingsEnum.NONE));
+            }
+            nonVisitedPostings = new PriorityQueue<>(nonVisitedOrds.size()) {
+                @Override
+                protected boolean lessThan(PostingsEnum a, PostingsEnum b) {
+                    return a.docID() < b.docID();
+                }
+            };
+            for (PostingsEnum pe : nonVisitedOrds.values()) {
+                nonVisitedPostings.add(pe);
+            }
+        }
+
+        void onVisitedOrdinal(long ordinal) throws IOException {
+            numNonVisitedOrds--;
+            if (nonVisitedOrds == null) {
+                if (numNonVisitedOrds <= MAX_TERMS_FOR_DYNAMIC_PRUNING) {
+                    startPruning();
+                }
+            } else {
+                if (nonVisitedOrds.remove(ordinal) != null) {
+                    // Could we make this more efficient?
+                    nonVisitedPostings.clear();
+                    for (PostingsEnum pe : nonVisitedOrds.values()) {
+                        nonVisitedPostings.add(pe);
+                    }
+                }
+            }
+        }
     }
 
     @Override
     public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException {
         values = valuesSource.globalOrdinalsValues(aggCtx.getLeafReaderContext());
+
+        if (parent == null && field != null) {
+            // This optimization only applies to top-level cardinality aggregations that apply to fields indexed with an inverted index.
+            final Terms indexTerms = aggCtx.getLeafReaderContext().reader().terms(field);
+            if (indexTerms != null) {
+                BitArray bits = visitedOrds.get(0);
+                final int numNonVisitedOrds = maxOrd - (bits == null ? 0 : (int) bits.cardinality());
+                if (maxOrd <= MAX_FIELD_CARDINALITY_FOR_DYNAMIC_PRUNING || numNonVisitedOrds <= MAX_TERMS_FOR_DYNAMIC_PRUNING) {
+                    dynamicPruningAttempts++;
+                    return new LeafBucketCollector() {
+
+                        final BitArray bits;
+                        final CompetitiveIterator competitiveIterator;
+
+                        {
+                            // This optimization only works for top-level cardinality aggregations that collect bucket 0, so we can retrieve
+                            // the appropriate BitArray ahead of time.
+                            visitedOrds = bigArrays.grow(visitedOrds, 1);
+                            BitArray bits = visitedOrds.get(0);
+                            if (bits == null) {
+                                bits = new BitArray(maxOrd, bigArrays);
+                                visitedOrds.set(0, bits);
+                            }
+                            this.bits = bits;
+                            final DocIdSetIterator docsWithField = valuesSource.ordinalsValues(aggCtx.getLeafReaderContext());
+                            competitiveIterator = new CompetitiveIterator(numNonVisitedOrds, bits, indexTerms, docsWithField);
+                            if (numNonVisitedOrds <= MAX_TERMS_FOR_DYNAMIC_PRUNING) {
+                                competitiveIterator.startPruning();
+                            }
+                        }
+
+                        @Override
+                        public void collect(int doc, long bucketOrd) throws IOException {
+                            if (values.advanceExact(doc)) {
+                                for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
+                                    if (bits.getAndSet(ord) == false) {
+                                        competitiveIterator.onVisitedOrdinal(ord);
+                                    }
+                                }
+                            }
+                        }
+
+                        @Override
+                        public CompetitiveIterator competitiveIterator() {
+                            return competitiveIterator;
+                        }
+                    };
+                }
+            } else {
+                final FieldInfo fi = aggCtx.getLeafReaderContext().reader().getFieldInfos().fieldInfo(field);
+                if (fi == null) {
+                    // The field doesn't exist at all, we can skip the segment entirely
+                    noData++;
+                    return LeafBucketCollector.NO_OP_COLLECTOR;
+                } else if (fi.getIndexOptions() != IndexOptions.NONE) {
+                    // The field doesn't have terms while index options are not NONE. This means that this segment doesn't have a single
+                    // value for the field.
+                    noData++;
+                    return LeafBucketCollector.NO_OP_COLLECTOR;
+                }
+                // Otherwise we might be aggregating e.g. an IP field, which indexes data using points rather than an inverted index.
+            }
+        }
+
+        bruteForce++;
         return new LeafBucketCollector() {
             @Override
             public void collect(int doc, long bucketOrd) throws IOException {
@@ -157,4 +353,13 @@ public class GlobalOrdCardinalityAggregator extends NumericMetricsAggregator.Sin
         }
         Releasables.close(visitedOrds, counts);
     }
+
+    @Override
+    public void collectDebugInfo(BiConsumer<String, Object> add) {
+        super.collectDebugInfo(add);
+        add.accept("dynamic_pruning_attempted", dynamicPruningAttempts);
+        add.accept("dynamic_pruning_used", dynamicPruningSuccess);
+        add.accept("brute_force_used", bruteForce);
+        add.accept("skipped_due_to_no_data", noData);
+    }
 }

+ 17 - 0
server/src/test/java/org/elasticsearch/common/util/BitArrayTests.java

@@ -154,4 +154,21 @@ public class BitArrayTests extends ESTestCase {
             }
         }
     }
+
+    public void testGetAndSet() {
+        try (BitArray bitArray = new BitArray(1, BigArrays.NON_RECYCLING_INSTANCE)) {
+            assertFalse(bitArray.getAndSet(100));
+            assertFalse(bitArray.getAndSet(1000));
+            assertTrue(bitArray.getAndSet(100));
+            assertFalse(bitArray.getAndSet(101));
+            assertFalse(bitArray.getAndSet(999));
+            assertTrue(bitArray.getAndSet(1000));
+            assertFalse(bitArray.get(99));
+            assertTrue(bitArray.get(100));
+            assertTrue(bitArray.get(101));
+            assertTrue(bitArray.get(999));
+            assertTrue(bitArray.get(1000));
+            assertFalse(bitArray.get(1001));
+        }
+    }
 }

+ 311 - 0
server/src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregatorTests.java

@@ -9,20 +9,27 @@
 package org.elasticsearch.search.aggregations.metrics;
 
 import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.InetAddressPoint;
 import org.apache.lucene.document.IntPoint;
 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.document.StringField;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.search.FieldExistsQuery;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.tests.index.RandomIndexWriter;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.common.geo.GeoPoint;
+import org.elasticsearch.common.network.InetAddresses;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.CheckedConsumer;
 import org.elasticsearch.index.fielddata.ScriptDocValues;
+import org.elasticsearch.index.mapper.IpFieldMapper;
 import org.elasticsearch.index.mapper.KeywordFieldMapper;
 import org.elasticsearch.index.mapper.MappedFieldType;
 import org.elasticsearch.index.mapper.NumberFieldMapper;
@@ -36,6 +43,7 @@ import org.elasticsearch.script.ScriptService;
 import org.elasticsearch.script.ScriptType;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
+import org.elasticsearch.search.aggregations.Aggregator;
 import org.elasticsearch.search.aggregations.AggregatorTestCase;
 import org.elasticsearch.search.aggregations.InternalAggregation;
 import org.elasticsearch.search.aggregations.bucket.global.Global;
@@ -47,6 +55,8 @@ import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
 import org.elasticsearch.search.aggregations.support.ValuesSourceType;
 
 import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -58,6 +68,9 @@ import java.util.function.Function;
 
 import static java.util.Collections.emptyMap;
 import static java.util.Collections.singleton;
+import static org.elasticsearch.test.MapMatcher.assertMap;
+import static org.elasticsearch.test.MapMatcher.matchesMap;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 
 public class CardinalityAggregatorTests extends AggregatorTestCase {
 
@@ -218,6 +231,120 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
         }, mappedFieldTypes);
     }
 
+    public void testIndexedSingleValuedString() throws IOException {
+        // Indexing enables dynamic pruning optimizations
+        final CardinalityAggregationBuilder aggregationBuilder = new CardinalityAggregationBuilder("name").field("str_value");
+        final MappedFieldType mappedFieldTypes = new KeywordFieldMapper.KeywordFieldType("str_value");
+
+        CheckedConsumer<RandomIndexWriter, IOException> buildIndex = iw -> {
+            iw.addDocument(
+                Arrays.asList(
+                    new StringField("str_value", "one", Field.Store.NO),
+                    new SortedDocValuesField("str_value", new BytesRef("one"))
+                )
+            );
+            iw.addDocument(
+                Arrays.asList(
+                    new StringField("unrelatedField", "two", Field.Store.NO),
+                    new SortedDocValuesField("unrelatedField", new BytesRef("two"))
+                )
+            );
+            iw.addDocument(
+                Arrays.asList(
+                    new StringField("str_value", "three", Field.Store.NO),
+                    new SortedDocValuesField("str_value", new BytesRef("three"))
+                )
+            );
+            iw.addDocument(
+                Arrays.asList(
+                    new StringField("str_value", "one", Field.Store.NO),
+                    new SortedDocValuesField("str_value", new BytesRef("one"))
+                )
+            );
+        };
+
+        testAggregation(aggregationBuilder, new MatchAllDocsQuery(), buildIndex, card -> {
+            assertEquals(2, card.getValue(), 0);
+            assertTrue(AggregationInspectionHelper.hasValue(card));
+        }, mappedFieldTypes);
+
+        // Enforce auto-detection of the execution mode
+        aggregationBuilder.executionHint(null);
+        debugTestCase(
+            aggregationBuilder,
+            new MatchAllDocsQuery(),
+            buildIndex,
+            (InternalCardinality card, Class<? extends Aggregator> impl, Map<String, Map<String, Object>> debug) -> {
+                assertEquals(2, card.getValue(), 0);
+                assertEquals(GlobalOrdCardinalityAggregator.class, impl);
+                assertMap(
+                    debug,
+                    matchesMap().entry(
+                        "name",
+                        matchesMap().entry("dynamic_pruning_used", greaterThanOrEqualTo(1))
+                            .entry("dynamic_pruning_attempted", greaterThanOrEqualTo(1))
+                            .entry("skipped_due_to_no_data", 0)
+                            .entry("brute_force_used", 0)
+                    )
+                );
+            },
+            mappedFieldTypes
+        );
+    }
+
+    public void testIndexedSingleValuedIP() throws IOException {
+        // IP addresses are interesting to test because they use sorted doc values like keywords, but index data using points rather than an
+        // inverted index, so this triggers a different code path to disable dynamic pruning
+        final CardinalityAggregationBuilder aggregationBuilder = new CardinalityAggregationBuilder("name").field("ip_value");
+        final MappedFieldType mappedFieldTypes = new IpFieldMapper.IpFieldType("ip_value");
+
+        CheckedConsumer<RandomIndexWriter, IOException> buildIndex = iw -> {
+            InetAddress value = InetAddresses.forString("::1");
+            byte[] encodedValue = InetAddressPoint.encode(value);
+            iw.addDocument(
+                Arrays.asList(new InetAddressPoint("ip_value", value), new SortedDocValuesField("ip_value", new BytesRef(encodedValue)))
+            );
+            value = InetAddresses.forString("192.168.0.1");
+            encodedValue = InetAddressPoint.encode(value);
+            iw.addDocument(
+                Arrays.asList(new InetAddressPoint("ip_value", value), new SortedDocValuesField("ip_value", new BytesRef(encodedValue)))
+            );
+            value = InetAddresses.forString("::1");
+            encodedValue = InetAddressPoint.encode(value);
+            iw.addDocument(
+                Arrays.asList(new InetAddressPoint("ip_value", value), new SortedDocValuesField("ip_value", new BytesRef(encodedValue)))
+            );
+        };
+
+        testAggregation(aggregationBuilder, new MatchAllDocsQuery(), buildIndex, card -> {
+            assertEquals(2, card.getValue(), 0);
+            assertTrue(AggregationInspectionHelper.hasValue(card));
+        }, mappedFieldTypes);
+
+        // Enforce auto-detection of the execution mode
+        aggregationBuilder.executionHint(null);
+        debugTestCase(
+            aggregationBuilder,
+            new MatchAllDocsQuery(),
+            buildIndex,
+            (InternalCardinality card, Class<? extends Aggregator> impl, Map<String, Map<String, Object>> debug) -> {
+                assertEquals(2, card.getValue(), 0);
+                assertEquals(GlobalOrdCardinalityAggregator.class, impl);
+                assertMap(
+                    debug,
+                    matchesMap().entry(
+                        "name",
+                        matchesMap().entry("dynamic_pruning_used", 0)
+                            .entry("dynamic_pruning_attempted", 0)
+                            .entry("skipped_due_to_no_data", 0)
+                            .entry("brute_force_used", greaterThanOrEqualTo(1))
+                    )
+                );
+            },
+            mappedFieldTypes
+        );
+    }
+
     public void testSingleValuedStringValueScript() throws IOException {
         final CardinalityAggregationBuilder aggregationBuilder = new CardinalityAggregationBuilder("name").field("str_value")
             .script(new Script(ScriptType.INLINE, MockScriptEngine.NAME, "_value", emptyMap()));
@@ -377,6 +504,83 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
         }, mappedFieldTypes);
     }
 
+    public void testIndexedMultiValuedString() throws IOException {
+        // Indexing enables dynamic pruning optimizations
+        final CardinalityAggregationBuilder aggregationBuilder = new CardinalityAggregationBuilder("name").field("str_values");
+        final MappedFieldType mappedFieldTypes = new KeywordFieldMapper.KeywordFieldType("str_values");
+
+        CheckedConsumer<RandomIndexWriter, IOException> buildIndex = iw -> {
+            iw.addDocument(
+                List.of(
+                    new StringField("str_values", "one", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("one")),
+                    new StringField("str_values", "two", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("two"))
+                )
+            );
+            iw.addDocument(
+                List.of(
+                    new StringField("str_values", "one", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("one")),
+                    new StringField("str_values", "three", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("three"))
+                )
+            );
+            iw.addDocument(
+                List.of(
+                    new StringField("str_values", "three", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("three")),
+                    new StringField("str_values", "two", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("two"))
+                )
+            );
+            iw.addDocument(
+                List.of(
+                    new StringField("str_values", "three", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("three")),
+                    new StringField("str_values", "two", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("two"))
+                )
+            );
+            iw.addDocument(
+                List.of(
+                    new StringField("str_values", "two", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("two")),
+                    new StringField("str_values", "three", Field.Store.NO),
+                    new SortedSetDocValuesField("str_values", new BytesRef("three"))
+                )
+            );
+        };
+
+        testAggregation(aggregationBuilder, new MatchAllDocsQuery(), buildIndex, card -> {
+            assertEquals(3, card.getValue(), 0);
+            assertTrue(AggregationInspectionHelper.hasValue(card));
+        }, mappedFieldTypes);
+
+        // Enforce auto-detection of the execution mode
+        aggregationBuilder.executionHint(null);
+        debugTestCase(
+            aggregationBuilder,
+            new MatchAllDocsQuery(),
+            buildIndex,
+            (InternalCardinality card, Class<? extends Aggregator> impl, Map<String, Map<String, Object>> debug) -> {
+                assertEquals(3, card.getValue(), 0);
+                assertEquals(GlobalOrdCardinalityAggregator.class, impl);
+                assertMap(
+                    debug,
+                    matchesMap().entry(
+                        "name",
+                        matchesMap().entry("dynamic_pruning_used", greaterThanOrEqualTo(1))
+                            .entry("dynamic_pruning_attempted", greaterThanOrEqualTo(1))
+                            .entry("skipped_due_to_no_data", 0)
+                            .entry("brute_force_used", 0)
+                    )
+                );
+            },
+            mappedFieldTypes
+        );
+    }
+
     public void testUnmappedMissingString() throws IOException {
         CardinalityAggregationBuilder aggregationBuilder = new CardinalityAggregationBuilder("name").field("number").missing("🍌🍌🍌");
 
@@ -593,6 +797,113 @@ public class CardinalityAggregatorTests extends AggregatorTestCase {
         }, new AggTestConfig(aggregationBuilder, mappedFieldTypes));
     }
 
+    public void testIndexedWithMissingValues() throws IOException {
+        // Indexing enables dynamic pruning optimizations
+        final CardinalityAggregationBuilder aggregationBuilder = new CardinalityAggregationBuilder("name").field("str_value");
+        final MappedFieldType mappedFieldTypes = new KeywordFieldMapper.KeywordFieldType("str_value");
+
+        testAggregation(aggregationBuilder, new MatchAllDocsQuery(), iw -> {
+            iw.addDocument(Collections.emptySet());
+            iw.addDocument(Collections.emptySet());
+            iw.addDocument(
+                Arrays.asList(
+                    new StringField("str_value", "one", Field.Store.NO),
+                    new SortedDocValuesField("str_value", new BytesRef("one"))
+                )
+            );
+            iw.addDocument(Collections.emptySet());
+            iw.addDocument(
+                Arrays.asList(
+                    new StringField("unrelatedField", "two", Field.Store.NO),
+                    new SortedDocValuesField("unrelatedField", new BytesRef("two"))
+                )
+            );
+            iw.addDocument(Collections.emptySet());
+            iw.addDocument(Collections.emptySet());
+            iw.addDocument(
+                Arrays.asList(
+                    new StringField("str_value", "three", Field.Store.NO),
+                    new SortedDocValuesField("str_value", new BytesRef("three"))
+                )
+            );
+            iw.addDocument(Collections.emptySet());
+            iw.addDocument(Collections.emptySet());
+            iw.addDocument(Collections.emptySet());
+            iw.addDocument(
+                Arrays.asList(
+                    new StringField("str_value", "one", Field.Store.NO),
+                    new SortedDocValuesField("str_value", new BytesRef("one"))
+                )
+            );
+        }, card -> {
+            assertEquals(2, card.getValue(), 0);
+            assertTrue(AggregationInspectionHelper.hasValue(card));
+        }, mappedFieldTypes);
+    }
+
+    public void testMoreThan128UniqueStringValues() throws IOException {
+        // Indexing enables dynamic pruning optimizations
+        // Fields with more than 128 unique values exercise slightly different code paths
+        final CardinalityAggregationBuilder aggregationBuilder = new CardinalityAggregationBuilder("name").field("str_value");
+        final MappedFieldType mappedFieldTypes = new KeywordFieldMapper.KeywordFieldType("str_value");
+
+        CheckedConsumer<RandomIndexWriter, IOException> buildIndex = iw -> {
+            for (int i = 0; i < 200; ++i) {
+                final String value = Integer.toString(i);
+                iw.addDocument(
+                    Arrays.asList(
+                        new StringField("keyword1", Integer.toString(i % 2), Field.Store.NO),
+                        new StringField("keyword2", Integer.toString(i % 5), Field.Store.NO),
+                        new StringField("str_value", value, Field.Store.NO),
+                        new SortedDocValuesField("str_value", new BytesRef(value))
+                    )
+                );
+            }
+        };
+
+        debugTestCase(
+            aggregationBuilder,
+            new TermQuery(new Term("keyword1", "0")),
+            buildIndex,
+            (InternalCardinality card, Class<? extends Aggregator> impl, Map<String, Map<String, Object>> debug) -> {
+                assertEquals(100, card.getValue(), 0);
+                assertEquals(GlobalOrdCardinalityAggregator.class, impl);
+                assertMap(
+                    debug,
+                    matchesMap().entry(
+                        "name",
+                        matchesMap().entry("dynamic_pruning_used", greaterThanOrEqualTo(1))
+                            .entry("dynamic_pruning_attempted", greaterThanOrEqualTo(1))
+                            .entry("skipped_due_to_no_data", 0)
+                            .entry("brute_force_used", 0)
+                    )
+                );
+            },
+            mappedFieldTypes
+        );
+
+        debugTestCase(
+            aggregationBuilder,
+            new TermQuery(new Term("keyword2", "0")),
+            buildIndex,
+            (InternalCardinality card, Class<? extends Aggregator> impl, Map<String, Map<String, Object>> debug) -> {
+                assertEquals(40, card.getValue(), 0);
+                assertEquals(GlobalOrdCardinalityAggregator.class, impl);
+                assertMap(
+                    debug,
+                    matchesMap().entry(
+                        "name",
+                        matchesMap().entry("dynamic_pruning_used", 0)
+                            .entry("dynamic_pruning_attempted", greaterThanOrEqualTo(1))
+                            .entry("skipped_due_to_no_data", 0)
+                            .entry("brute_force_used", 0)
+                    )
+                );
+            },
+            mappedFieldTypes
+        );
+    }
+
     private void testAggregation(
         Query query,
         CheckedConsumer<RandomIndexWriter, IOException> buildIndex,