You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gs...@apache.org on 2021/11/23 14:00:17 UTC

[lucene] branch main updated: LUCENE-10062: Switch to numeric doc values for encoding taxonomy ordinals (#264)

This is an automated email from the ASF dual-hosted git repository.

gsmiller pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new 6ee69e0  LUCENE-10062: Switch to numeric doc values for encoding taxonomy ordinals (#264)
6ee69e0 is described below

commit 6ee69e06fbd147598b49a16aee89496ff072a0ef
Author: Greg Miller <gs...@gmail.com>
AuthorDate: Tue Nov 23 06:00:11 2021 -0800

    LUCENE-10062: Switch to numeric doc values for encoding taxonomy ordinals (#264)
---
 lucene/CHANGES.txt                                 |   9 +-
 lucene/MIGRATE.md                                  |  17 +++
 .../java/org/apache/lucene/facet/FacetsConfig.java |  65 ++-------
 .../java/org/apache/lucene/facet/package-info.java |   7 +-
 .../facet/taxonomy/CachedOrdinalsReader.java       | 161 ---------------------
 .../facet/taxonomy/DocValuesOrdinalsReader.java    | 123 ----------------
 .../facet/taxonomy/FastTaxonomyFacetCounts.java    |  46 ++----
 .../facet/taxonomy/OrdinalMappingLeafReader.java   | 119 +++++++--------
 .../lucene/facet/taxonomy/OrdinalsReader.java      |  43 ------
 .../lucene/facet/taxonomy/TaxonomyFacetCounts.java |  69 ---------
 .../lucene/facet/taxonomy/TaxonomyFacetLabels.java |  88 +++++++----
 .../TaxonomyFacetSumFloatAssociations.java         |   3 -
 .../taxonomy/TaxonomyFacetSumIntAssociations.java  |   3 -
 .../taxonomy/TaxonomyFacetSumValueSource.java      |  44 +++---
 .../directory/DirectoryTaxonomyWriter.java         |   2 -
 .../org/apache/lucene/facet/FacetTestCase.java     |  17 +--
 .../lucene/facet/TestMultipleIndexFields.java      |   2 +-
 .../facet/taxonomy/TestCachedOrdinalsReader.java   |  88 -----------
 .../facet/taxonomy/TestTaxonomyFacetCounts.java    |  11 +-
 .../taxonomy/TestTaxonomyFacetSumValueSource.java  |   3 +-
 20 files changed, 191 insertions(+), 729 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index cd7a040..27ba87f 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -395,6 +395,9 @@ Improvements
   See release notes. https://github.com/locationtech/spatial4j/releases/tag/spatial4j-0.8
   (David Smiley)
 
+* LUCENE-10062: Switch taxonomy faceting to use numeric doc values for storing ordinals instead of binary doc values
+  with its own custom encoding. (Greg Miller)
+
 Bug fixes
 ---------------------
 
@@ -434,14 +437,14 @@ Bug fixes
 
 * LUCENE-10047: Fix a value de-duping bug in LongValueFacetCounts and RangeFacetCounts
   (Greg Miller)
-  
+
 * LUCENE-10101, LUCENE-9281: Use getField() instead of getDeclaredField() to
   minimize security impact by analysis SPI discovery. (Uwe Schindler)
 
 * LUCENE-10114: Remove unused byte order mark in Lucene90PostingsWriter. This
   was initially introduced by accident in Lucene 8.4. (Uwe Schindler)
 
-* LUCENE-10140: Fix cases where minimizing interval iterators could return 
+* LUCENE-10140: Fix cases where minimizing interval iterators could return
   incorrect matches (Nikolay Khitrin, Alan Woodward)
 
 Changes in Backwards Compatibility Policy
@@ -461,7 +464,7 @@ Build
 
 * LUCENE-10195: Add gradle cache option and make some tasks cacheable. (Jerome Prinet, Dawid Weiss)
 
-* LUCENE-10198: LUCENE-10198: Allow external JAVA_OPTS in gradlew scripts; use sane defaults 
+* LUCENE-10198: LUCENE-10198: Allow external JAVA_OPTS in gradlew scripts; use sane defaults
   (balmukund.mandal@intel.com, Dawid Weiss)
 
 * LUCENE-10163: Move LICENSE and NOTICE files to top level to satisfy src artifact requirements (janhoy)
diff --git a/lucene/MIGRATE.md b/lucene/MIGRATE.md
index eb5436d..20bc551 100644
--- a/lucene/MIGRATE.md
+++ b/lucene/MIGRATE.md
@@ -450,3 +450,20 @@ structure. Use a standard BoostQuery here instead.
 
 Rather than using `setSort()` to change sort values, you should instead create
 a new Sort instance with the new values.
+
+## Taxonomy-based faceting uses more modern encodings (LUCENE-9450, LUCENE-10062, LUCENE-10122)
+
+The side-car taxonomy index now uses doc values for ord-to-path lookup (LUCENE-9450) and parent
+lookup (LUCENE-10122) instead of stored fields and positions (respectively). Document ordinals
+are now encoded with `SortedNumericDocValues` instead of using a custom (v-int) binary format.
+Performance gains have been observed with these encoding changes. These changes were introduced
+in 9.0, and 9.x releases remain backwards-compatible with 8.x indexes, but starting with 10.0,
+only the newer formats are supported. Users will need to create a new index with all their
+documents using 9.0 or later to pick up the new format and remain compatible with 10.x releases.
+Just re-adding documents to an existing index is not enough to pick up the changes as the
+format will "stick" to whatever version was used to initially create the index.
+
+Additionally, `OrdinalsReader` (and sub-classes) are fully removed starting with 10.0. These
+classes were `@Deprecated` starting with 9.0. Users are encouraged to rely on the default
+taxonomy facet encodings where possible. If custom formats are needed, users will need
+to manage the indexed data on their own and create new `Facet` implementations to use it.
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetsConfig.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetsConfig.java
index 7558f61..633033b 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/FacetsConfig.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetsConfig.java
@@ -28,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
@@ -409,9 +410,19 @@ public class FacetsConfig {
         indexDrillDownTerms(doc, indexFieldName, dimConfig, facetLabel);
       }
 
-      // Facet counts:
-      // DocValues are considered stored fields:
-      doc.add(new BinaryDocValuesField(indexFieldName, dedupAndEncode(ordinals.get())));
+      // Dedupe and encode the ordinals. It's not important that we sort here
+      // (SortedNumericDocValuesField will handle this internally), but we
+      // sort to identify dups (since SNDVF doesn't dedupe):
+      IntsRef ords = ordinals.get();
+      Arrays.sort(ords.ints, ords.offset, ords.offset + ords.length);
+      int prev = -1;
+      for (int i = 0; i < ords.length; i++) {
+        int ord = ords.ints[ords.offset + i];
+        if (ord > prev) {
+          doc.add(new SortedNumericDocValuesField(indexFieldName, ord));
+          prev = ord;
+        }
+      }
     }
   }
 
@@ -507,54 +518,6 @@ public class FacetsConfig {
     }
   }
 
-  /** Encodes ordinals into a BytesRef; expert: subclass can override this to change encoding. */
-  protected BytesRef dedupAndEncode(IntsRef ordinals) {
-    Arrays.sort(ordinals.ints, ordinals.offset, ordinals.length);
-    byte[] bytes = new byte[5 * ordinals.length];
-    int lastOrd = -1;
-    int upto = 0;
-    for (int i = 0; i < ordinals.length; i++) {
-      int ord = ordinals.ints[ordinals.offset + i];
-      // ord could be == lastOrd, so we must dedup:
-      if (ord > lastOrd) {
-        int delta;
-        if (lastOrd == -1) {
-          delta = ord;
-        } else {
-          delta = ord - lastOrd;
-        }
-        if ((delta & ~0x7F) == 0) {
-          bytes[upto] = (byte) delta;
-          upto++;
-        } else if ((delta & ~0x3FFF) == 0) {
-          bytes[upto] = (byte) (0x80 | ((delta & 0x3F80) >> 7));
-          bytes[upto + 1] = (byte) (delta & 0x7F);
-          upto += 2;
-        } else if ((delta & ~0x1FFFFF) == 0) {
-          bytes[upto] = (byte) (0x80 | ((delta & 0x1FC000) >> 14));
-          bytes[upto + 1] = (byte) (0x80 | ((delta & 0x3F80) >> 7));
-          bytes[upto + 2] = (byte) (delta & 0x7F);
-          upto += 3;
-        } else if ((delta & ~0xFFFFFFF) == 0) {
-          bytes[upto] = (byte) (0x80 | ((delta & 0xFE00000) >> 21));
-          bytes[upto + 1] = (byte) (0x80 | ((delta & 0x1FC000) >> 14));
-          bytes[upto + 2] = (byte) (0x80 | ((delta & 0x3F80) >> 7));
-          bytes[upto + 3] = (byte) (delta & 0x7F);
-          upto += 4;
-        } else {
-          bytes[upto] = (byte) (0x80 | ((delta & 0xF0000000) >> 28));
-          bytes[upto + 1] = (byte) (0x80 | ((delta & 0xFE00000) >> 21));
-          bytes[upto + 2] = (byte) (0x80 | ((delta & 0x1FC000) >> 14));
-          bytes[upto + 3] = (byte) (0x80 | ((delta & 0x3F80) >> 7));
-          bytes[upto + 4] = (byte) (delta & 0x7F);
-          upto += 5;
-        }
-        lastOrd = ord;
-      }
-    }
-    return new BytesRef(bytes, 0, upto);
-  }
-
   private void checkTaxoWriter(TaxonomyWriter taxoWriter) {
     if (taxoWriter == null) {
       throw new IllegalStateException(
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
index a0c64ae..9adc9dc 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/package-info.java
@@ -23,10 +23,9 @@
  * <ul>
  *   <li>Taxonomy-based methods rely on a separate taxonomy index to map hierarchical facet paths to
  *       global int ordinals for fast counting at search time; these methods can compute counts
- *       (({@link org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts}, {@link
- *       org.apache.lucene.facet.taxonomy.TaxonomyFacetCounts}) aggregate long or double values
- *       {@link org.apache.lucene.facet.taxonomy.TaxonomyFacetSumIntAssociations}, {@link
- *       org.apache.lucene.facet.taxonomy.TaxonomyFacetSumFloatAssociations}, {@link
+ *       (({@link org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts}) aggregate long or
+ *       double values {@link org.apache.lucene.facet.taxonomy.TaxonomyFacetSumIntAssociations},
+ *       {@link org.apache.lucene.facet.taxonomy.TaxonomyFacetSumFloatAssociations}, {@link
  *       org.apache.lucene.facet.taxonomy.TaxonomyFacetSumValueSource}. Add {@link
  *       org.apache.lucene.facet.FacetField} or {@link
  *       org.apache.lucene.facet.taxonomy.AssociationFacetField} to your documents at index time to
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CachedOrdinalsReader.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CachedOrdinalsReader.java
deleted file mode 100644
index 0729c90..0000000
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CachedOrdinalsReader.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.facet.taxonomy;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Map;
-import java.util.WeakHashMap;
-import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Accountables;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.IntsRef;
-import org.apache.lucene.util.RamUsageEstimator;
-
-/**
- * A per-segment cache of documents' facet ordinals. Every {@link CachedOrds} holds the ordinals in
- * a raw {@code int[]}, and therefore consumes as much RAM as the total number of ordinals found in
- * the segment, but saves the CPU cost of decoding ordinals during facet counting.
- *
- * <p><b>NOTE:</b> every {@link CachedOrds} is limited to 2.1B total ordinals. If that is a
- * limitation for you then consider limiting the segment size to fewer documents, or use an
- * alternative cache which pages through the category ordinals.
- *
- * <p><b>NOTE:</b> when using this cache, it is advised to use a {@link DocValuesFormat} that does
- * not cache the data in memory, at least for the category lists fields, or otherwise you'll be
- * doing double-caching.
- *
- * <p><b>NOTE:</b> create one instance of this and re-use it for all facet implementations (the
- * cache is per-instance, not static).
- */
-public class CachedOrdinalsReader extends OrdinalsReader implements Accountable {
-
-  private final OrdinalsReader source;
-
-  private final Map<Object, CachedOrds> ordsCache = new WeakHashMap<>();
-
-  /** Sole constructor. */
-  public CachedOrdinalsReader(OrdinalsReader source) {
-    this.source = source;
-  }
-
-  private synchronized CachedOrds getCachedOrds(LeafReaderContext context) throws IOException {
-    IndexReader.CacheHelper cacheHelper = context.reader().getCoreCacheHelper();
-    if (cacheHelper == null) {
-      throw new IllegalStateException("Cannot cache ordinals on leaf: " + context.reader());
-    }
-    Object cacheKey = cacheHelper.getKey();
-    CachedOrds ords = ordsCache.get(cacheKey);
-    if (ords == null) {
-      ords = new CachedOrds(source.getReader(context), context.reader().maxDoc());
-      ordsCache.put(cacheKey, ords);
-    }
-
-    return ords;
-  }
-
-  @Override
-  public String getIndexFieldName() {
-    return source.getIndexFieldName();
-  }
-
-  @Override
-  public OrdinalsSegmentReader getReader(LeafReaderContext context) throws IOException {
-    final CachedOrds cachedOrds = getCachedOrds(context);
-    return new OrdinalsSegmentReader() {
-      @Override
-      public void get(int docID, IntsRef ordinals) {
-        ordinals.ints = cachedOrds.ordinals;
-        ordinals.offset = cachedOrds.offsets[docID];
-        ordinals.length = cachedOrds.offsets[docID + 1] - ordinals.offset;
-      }
-    };
-  }
-
-  /** Holds the cached ordinals in two parallel {@code int[]} arrays. */
-  public static final class CachedOrds implements Accountable {
-
-    /** Index into {@link #ordinals} for each document. */
-    public final int[] offsets;
-
-    /** Holds ords for all docs. */
-    public final int[] ordinals;
-
-    /**
-     * Creates a new {@link CachedOrds} from the {@link BinaryDocValues}. Assumes that the {@link
-     * BinaryDocValues} is not {@code null}.
-     */
-    public CachedOrds(OrdinalsSegmentReader source, int maxDoc) throws IOException {
-      offsets = new int[maxDoc + 1];
-      int[] ords = new int[maxDoc]; // let's assume one ordinal per-document as an initial size
-
-      // this aggregator is limited to Integer.MAX_VALUE total ordinals.
-      long totOrds = 0;
-      final IntsRef values = new IntsRef(32);
-      for (int docID = 0; docID < maxDoc; docID++) {
-        offsets[docID] = (int) totOrds;
-        source.get(docID, values);
-        long nextLength = totOrds + values.length;
-        if (nextLength > ords.length) {
-          if (nextLength > ArrayUtil.MAX_ARRAY_LENGTH) {
-            throw new IllegalStateException("too many ordinals (>= " + nextLength + ") to cache");
-          }
-          ords = ArrayUtil.grow(ords, (int) nextLength);
-        }
-        System.arraycopy(values.ints, 0, ords, (int) totOrds, values.length);
-        totOrds = nextLength;
-      }
-      offsets[maxDoc] = (int) totOrds;
-
-      // if ords array is bigger by more than 10% of what we really need, shrink it
-      if ((double) totOrds / ords.length < 0.9) {
-        this.ordinals = new int[(int) totOrds];
-        System.arraycopy(ords, 0, this.ordinals, 0, (int) totOrds);
-      } else {
-        this.ordinals = ords;
-      }
-    }
-
-    @Override
-    public long ramBytesUsed() {
-      long mem = RamUsageEstimator.shallowSizeOf(this) + RamUsageEstimator.sizeOf(offsets);
-      if (offsets != ordinals) {
-        mem += RamUsageEstimator.sizeOf(ordinals);
-      }
-      return mem;
-    }
-  }
-
-  @Override
-  public synchronized long ramBytesUsed() {
-    long bytes = 0;
-    for (CachedOrds ords : ordsCache.values()) {
-      bytes += ords.ramBytesUsed();
-    }
-
-    return bytes;
-  }
-
-  @Override
-  public synchronized Collection<Accountable> getChildResources() {
-    return Accountables.namedAccountables("segment", ordsCache);
-  }
-}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/DocValuesOrdinalsReader.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/DocValuesOrdinalsReader.java
deleted file mode 100644
index 7099de7..0000000
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/DocValuesOrdinalsReader.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.facet.taxonomy;
-
-import java.io.IOException;
-import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRef;
-
-/** Decodes ordinals previously indexed into a BinaryDocValues field */
-public class DocValuesOrdinalsReader extends OrdinalsReader {
-  private final String field;
-
-  /** Default constructor. */
-  public DocValuesOrdinalsReader() {
-    this(FacetsConfig.DEFAULT_INDEX_FIELD_NAME);
-  }
-
-  /** Create this, with the specified indexed field name. */
-  public DocValuesOrdinalsReader(String field) {
-    this.field = field;
-  }
-
-  @Override
-  public OrdinalsSegmentReader getReader(LeafReaderContext context) throws IOException {
-    BinaryDocValues values0 = context.reader().getBinaryDocValues(field);
-    if (values0 == null) {
-      values0 = DocValues.emptyBinary();
-    }
-
-    final BinaryDocValues values = values0;
-
-    return new OrdinalsSegmentReader() {
-
-      private int lastDocID;
-
-      @Override
-      public void get(int docID, IntsRef ordinals) throws IOException {
-        if (docID < lastDocID) {
-          throw new AssertionError(
-              "docs out of order: lastDocID=" + lastDocID + " vs docID=" + docID);
-        }
-        lastDocID = docID;
-        if (docID > values.docID()) {
-          values.advance(docID);
-        }
-        final BytesRef bytes;
-        if (values.docID() == docID) {
-          bytes = values.binaryValue();
-        } else {
-          bytes = new BytesRef(BytesRef.EMPTY_BYTES);
-        }
-        decode(bytes, ordinals);
-      }
-    };
-  }
-
-  @Override
-  public String getIndexFieldName() {
-    return field;
-  }
-
-  /**
-   * Subclass and override if you change the encoding. The method is marked 'public' to allow
-   * decoding of binary payload containing ordinals without instantiating an {@link
-   * org.apache.lucene.facet.taxonomy.OrdinalsReader.OrdinalsSegmentReader}.
-   *
-   * <p>This takes care of use cases where an application instantiates {@link
-   * org.apache.lucene.index.BinaryDocValues} reader for a facet field outside this class, reads the
-   * binary payload for a document and decodes the ordinals in the payload.
-   *
-   * @param buf binary payload containing encoded ordinals
-   * @param ordinals buffer for decoded ordinals
-   */
-  public void decode(BytesRef buf, IntsRef ordinals) {
-
-    // grow the buffer up front, even if by a large number of values (buf.length)
-    // that saves the need to check inside the loop for every decoded value if
-    // the buffer needs to grow.
-    if (ordinals.ints.length < buf.length) {
-      ordinals.ints = ArrayUtil.grow(ordinals.ints, buf.length);
-    }
-
-    ordinals.offset = 0;
-    ordinals.length = 0;
-
-    // it is better if the decoding is inlined like so, and not e.g.
-    // in a utility method
-    int upto = buf.offset + buf.length;
-    int value = 0;
-    int offset = buf.offset;
-    int prev = 0;
-    while (offset < upto) {
-      byte b = buf.bytes[offset++];
-      if (b >= 0) {
-        ordinals.ints[ordinals.length] = ((value << 7) | b) + prev;
-        value = 0;
-        prev = ordinals.ints[ordinals.length];
-        ordinals.length++;
-      } else {
-        value = (value << 7) | (b & 0x7F);
-      }
-    }
-  }
-}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FastTaxonomyFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FastTaxonomyFacetCounts.java
index 4b6332b..c3b738c 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FastTaxonomyFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FastTaxonomyFacetCounts.java
@@ -22,14 +22,13 @@ import java.util.List;
 import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.search.ConjunctionUtils;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 
 /**
  * Computes facets counts, assuming the default encoding into DocValues was used.
@@ -70,8 +69,8 @@ public class FastTaxonomyFacetCounts extends IntTaxonomyFacets {
 
   private final void count(List<MatchingDocs> matchingDocs) throws IOException {
     for (MatchingDocs hits : matchingDocs) {
-      BinaryDocValues dv = hits.context.reader().getBinaryDocValues(indexFieldName);
-      if (dv == null) { // this reader does not have DocValues for the requested category list
+      SortedNumericDocValues dv = hits.context.reader().getSortedNumericDocValues(indexFieldName);
+      if (dv == null) {
         continue;
       }
 
@@ -79,21 +78,8 @@ public class FastTaxonomyFacetCounts extends IntTaxonomyFacets {
           ConjunctionUtils.intersectIterators(Arrays.asList(hits.bits.iterator(), dv));
 
       for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
-        final BytesRef bytesRef = dv.binaryValue();
-        byte[] bytes = bytesRef.bytes;
-        int end = bytesRef.offset + bytesRef.length;
-        int ord = 0;
-        int offset = bytesRef.offset;
-        int prev = 0;
-        while (offset < end) {
-          byte b = bytes[offset++];
-          if (b >= 0) {
-            prev = ord = ((ord << 7) | b) + prev;
-            increment(ord);
-            ord = 0;
-          } else {
-            ord = (ord << 7) | (b & 0x7F);
-          }
+        for (int i = 0; i < dv.docValueCount(); i++) {
+          increment((int) dv.nextValue());
         }
       }
     }
@@ -103,8 +89,8 @@ public class FastTaxonomyFacetCounts extends IntTaxonomyFacets {
 
   private final void countAll(IndexReader reader) throws IOException {
     for (LeafReaderContext context : reader.leaves()) {
-      BinaryDocValues dv = context.reader().getBinaryDocValues(indexFieldName);
-      if (dv == null) { // this reader does not have DocValues for the requested category list
+      SortedNumericDocValues dv = context.reader().getSortedNumericDocValues(indexFieldName);
+      if (dv == null) {
         continue;
       }
 
@@ -114,21 +100,9 @@ public class FastTaxonomyFacetCounts extends IntTaxonomyFacets {
         if (liveDocs != null && liveDocs.get(doc) == false) {
           continue;
         }
-        final BytesRef bytesRef = dv.binaryValue();
-        byte[] bytes = bytesRef.bytes;
-        int end = bytesRef.offset + bytesRef.length;
-        int ord = 0;
-        int offset = bytesRef.offset;
-        int prev = 0;
-        while (offset < end) {
-          byte b = bytes[offset++];
-          if (b >= 0) {
-            prev = ord = ((ord << 7) | b) + prev;
-            increment(ord);
-            ord = 0;
-          } else {
-            ord = (ord << 7) | (b & 0x7F);
-          }
+
+        for (int i = 0; i < dv.docValueCount(); i++) {
+          increment((int) dv.nextValue());
         }
       }
     }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/OrdinalMappingLeafReader.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/OrdinalMappingLeafReader.java
index 731d4a9..90c73b7 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/OrdinalMappingLeafReader.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/OrdinalMappingLeafReader.java
@@ -16,19 +16,19 @@
  */
 package org.apache.lucene.facet.taxonomy;
 
+import com.carrotsearch.hppc.IntArrayList;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
 import org.apache.lucene.facet.FacetsConfig;
 import org.apache.lucene.facet.FacetsConfig.DimConfig;
-import org.apache.lucene.facet.taxonomy.OrdinalsReader.OrdinalsSegmentReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter.OrdinalMap;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.FilterBinaryDocValues;
 import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.FilterSortedNumericDocValues;
 import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
 
 /**
  * A {@link org.apache.lucene.index.FilterLeafReader} for updating facets ordinal references, based
@@ -62,53 +62,67 @@ import org.apache.lucene.util.IntsRef;
  */
 public class OrdinalMappingLeafReader extends FilterLeafReader {
 
-  // silly way, but we need to use dedupAndEncode and it's protected on FacetsConfig.
-  private static class InnerFacetsConfig extends FacetsConfig {
+  private class OrdinalMappingSortedNumericDocValues extends FilterSortedNumericDocValues {
+    private final IntArrayList currentValues;
+    private int currIndex;
 
-    InnerFacetsConfig() {}
+    OrdinalMappingSortedNumericDocValues(SortedNumericDocValues in) {
+      super(in);
+      currentValues = new IntArrayList(32);
+    }
 
     @Override
-    public BytesRef dedupAndEncode(IntsRef ordinals) {
-      return super.dedupAndEncode(ordinals);
+    public boolean advanceExact(int target) throws IOException {
+      boolean result = in.advanceExact(target);
+      if (result) {
+        reloadValues();
+      }
+      return result;
     }
-  }
-
-  private class OrdinalMappingBinaryDocValues extends FilterBinaryDocValues {
-
-    private final IntsRef ordinals = new IntsRef(32);
-    private final OrdinalsSegmentReader ordsReader;
 
-    OrdinalMappingBinaryDocValues(OrdinalsSegmentReader ordsReader, BinaryDocValues in)
-        throws IOException {
-      super(in);
-      this.ordsReader = ordsReader;
+    @Override
+    public int advance(int target) throws IOException {
+      int result = in.advance(target);
+      if (result != DocIdSetIterator.NO_MORE_DOCS) {
+        reloadValues();
+      }
+      return result;
     }
 
-    @SuppressWarnings("synthetic-access")
     @Override
-    public BytesRef binaryValue() {
-      try {
-        // NOTE: this isn't quite koscher, because in general
-        // multiple threads can call BinaryDV.get which would
-        // then conflict on the single ordinals instance, but
-        // because this impl is only used for merging, we know
-        // only 1 thread calls us:
-        ordsReader.get(docID(), ordinals);
+    public int nextDoc() throws IOException {
+      int result = in.nextDoc();
+      if (result != DocIdSetIterator.NO_MORE_DOCS) {
+        reloadValues();
+      }
+      return result;
+    }
 
-        // map the ordinals
-        for (int i = 0; i < ordinals.length; i++) {
-          ordinals.ints[i] = ordinalMap[ordinals.ints[i]];
-        }
+    @Override
+    public int docValueCount() {
+      return currentValues.elementsCount;
+    }
 
-        return encode(ordinals);
-      } catch (IOException e) {
-        throw new RuntimeException("error reading category ordinals for doc " + docID(), e);
+    private void reloadValues() throws IOException {
+      currIndex = 0;
+      currentValues.clear();
+      for (int i = 0; i < in.docValueCount(); i++) {
+        int originalOrd = Math.toIntExact(in.nextValue());
+        currentValues.add(ordinalMap[originalOrd]);
       }
+      Arrays.sort(currentValues.buffer, 0, currentValues.elementsCount);
+    }
+
+    @Override
+    public long nextValue() {
+      assert currIndex < currentValues.size();
+      int actual = currentValues.get(currIndex);
+      currIndex++;
+      return actual;
     }
   }
 
   private final int[] ordinalMap;
-  private final InnerFacetsConfig facetsConfig;
   private final Set<String> facetFields;
 
   /**
@@ -118,38 +132,27 @@ public class OrdinalMappingLeafReader extends FilterLeafReader {
   public OrdinalMappingLeafReader(LeafReader in, int[] ordinalMap, FacetsConfig srcConfig) {
     super(in);
     this.ordinalMap = ordinalMap;
-    facetsConfig = new InnerFacetsConfig();
     facetFields = new HashSet<>();
     for (DimConfig dc : srcConfig.getDimConfigs().values()) {
       facetFields.add(dc.indexFieldName);
     }
     // always add the default indexFieldName. This is because FacetsConfig does
     // not explicitly record dimensions that were indexed under the default
-    // DimConfig, unless they have a custome DimConfig.
+    // DimConfig, unless they have a custom DimConfig.
     facetFields.add(FacetsConfig.DEFAULT_DIM_CONFIG.indexFieldName);
   }
 
-  /**
-   * Expert: encodes category ordinals into a BytesRef. Override in case you use custom encoding,
-   * other than the default done by FacetsConfig.
-   */
-  protected BytesRef encode(IntsRef ordinals) {
-    return facetsConfig.dedupAndEncode(ordinals);
-  }
-
-  /** Expert: override in case you used custom encoding for the categories under this field. */
-  protected OrdinalsReader getOrdinalsReader(String field) {
-    return new DocValuesOrdinalsReader(field);
-  }
-
   @Override
-  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
-    if (facetFields.contains(field)) {
-      final OrdinalsReader ordsReader = getOrdinalsReader(field);
-      return new OrdinalMappingBinaryDocValues(
-          ordsReader.getReader(in.getContext()), in.getBinaryDocValues(field));
+  public SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
+    SortedNumericDocValues original = in.getSortedNumericDocValues(field);
+    if (original != null && facetFields.contains(field)) {
+      // The requested field is a facet ordinals field _and_ it's non-null, so move forward with
+      // mapping:
+      return new OrdinalMappingSortedNumericDocValues(original);
     } else {
-      return in.getBinaryDocValues(field);
+      // The requested field either isn't present (null) or isn't a facet ordinals field. Either
+      // way, just return the original:
+      return original;
     }
   }
 
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/OrdinalsReader.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/OrdinalsReader.java
deleted file mode 100644
index 5251109..0000000
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/OrdinalsReader.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.facet.taxonomy;
-
-import java.io.IOException;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.util.IntsRef;
-
-/** Provides per-document ordinals. */
-public abstract class OrdinalsReader {
-
-  /** Returns ordinals for documents in one segment. */
-  public abstract static class OrdinalsSegmentReader {
-    /** Get the ordinals for this document. ordinals.offset must always be 0! */
-    public abstract void get(int doc, IntsRef ordinals) throws IOException;
-
-    /** Default constructor. */
-    public OrdinalsSegmentReader() {}
-  }
-
-  /** Default constructor. */
-  public OrdinalsReader() {}
-
-  /** Set current atomic reader. */
-  public abstract OrdinalsSegmentReader getReader(LeafReaderContext context) throws IOException;
-
-  /** Returns the indexed field name this {@code OrdinalsReader} is reading from. */
-  public abstract String getIndexFieldName();
-}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetCounts.java
deleted file mode 100644
index 1b14b07..0000000
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetCounts.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.facet.taxonomy;
-
-import java.io.IOException;
-import java.util.List;
-import org.apache.lucene.facet.FacetsCollector;
-import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
-import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.IntsRef;
-
-/**
- * Reads from any {@link OrdinalsReader}; use {@link FastTaxonomyFacetCounts} if you are using the
- * default encoding from {@link BinaryDocValues}.
- *
- * @lucene.experimental
- */
-public class TaxonomyFacetCounts extends IntTaxonomyFacets {
-  private final OrdinalsReader ordinalsReader;
-
-  /**
-   * Create {@code TaxonomyFacetCounts}, which also counts all facet labels. Use this for a
-   * non-default {@link OrdinalsReader}; otherwise use {@link FastTaxonomyFacetCounts}.
-   */
-  public TaxonomyFacetCounts(
-      OrdinalsReader ordinalsReader,
-      TaxonomyReader taxoReader,
-      FacetsConfig config,
-      FacetsCollector fc)
-      throws IOException {
-    super(ordinalsReader.getIndexFieldName(), taxoReader, config, fc);
-    this.ordinalsReader = ordinalsReader;
-    count(fc.getMatchingDocs());
-  }
-
-  private final void count(List<MatchingDocs> matchingDocs) throws IOException {
-    IntsRef scratch = new IntsRef();
-    for (MatchingDocs hits : matchingDocs) {
-      OrdinalsReader.OrdinalsSegmentReader ords = ordinalsReader.getReader(hits.context);
-      DocIdSetIterator docs = hits.bits.iterator();
-
-      int doc;
-      while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-        ords.get(doc, scratch);
-        for (int i = 0; i < scratch.length; i++) {
-          increment(scratch.ints[scratch.offset + i]);
-        }
-      }
-    }
-
-    rollup();
-  }
-}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetLabels.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetLabels.java
index f0c66ba..75198f2 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetLabels.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetLabels.java
@@ -20,8 +20,9 @@ import static org.apache.lucene.facet.taxonomy.TaxonomyReader.INVALID_ORDINAL;
 import static org.apache.lucene.facet.taxonomy.TaxonomyReader.ROOT_ORDINAL;
 
 import java.io.IOException;
+import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.index.SortedNumericDocValues;
 
 /**
  * Utility class to easily retrieve previously indexed facet labels, allowing you to skip also
@@ -34,19 +35,16 @@ public class TaxonomyFacetLabels {
   /** {@code TaxonomyReader} provided to the constructor */
   private final TaxonomyReader taxoReader;
 
-  /**
-   * {@code OrdinalsReader} to decode ordinals previously indexed into the {@code BinaryDocValues}
-   * facet field
-   */
-  private final OrdinalsReader ordsReader;
+  /** field storing the taxonomy ordinals */
+  private final String indexFieldName;
 
   /**
    * Sole constructor. Do not close the provided {@link TaxonomyReader} while still using this
    * instance!
    */
-  public TaxonomyFacetLabels(TaxonomyReader taxoReader, String indexFieldName) throws IOException {
+  public TaxonomyFacetLabels(TaxonomyReader taxoReader, String indexFieldName) {
     this.taxoReader = taxoReader;
-    this.ordsReader = new DocValuesOrdinalsReader(indexFieldName);
+    this.indexFieldName = indexFieldName;
   }
 
   /**
@@ -62,7 +60,13 @@ public class TaxonomyFacetLabels {
    * @throws IOException when a low-level IO issue occurs
    */
   public FacetLabelReader getFacetLabelReader(LeafReaderContext readerContext) throws IOException {
-    return new FacetLabelReader(ordsReader, readerContext);
+    SortedNumericDocValues ordinalValues =
+        DocValues.getSortedNumeric(readerContext.reader(), indexFieldName);
+    if (ordinalValues == null) {
+      ordinalValues = DocValues.emptySortedNumeric();
+    }
+
+    return new FacetLabelReader(ordinalValues);
   }
 
   /**
@@ -71,18 +75,20 @@ public class TaxonomyFacetLabels {
    * @lucene.experimental
    */
   public class FacetLabelReader {
-    private final OrdinalsReader.OrdinalsSegmentReader ordinalsSegmentReader;
-    private final IntsRef decodedOrds = new IntsRef();
+    /** By default, we store taxonomy ordinals in SortedNumericDocValues field */
+    private final SortedNumericDocValues ordinalValues;
+
     private int currentDocId = -1;
-    private int currentPos = -1;
+    private boolean currentDocHasValues;
+    private int currentPos;
+    private int currentDocOrdinalCount;
 
     // Lazily set when nextFacetLabel(int docId, String facetDimension) is first called
     private int[] parents;
 
-    /** Sole constructor. */
-    public FacetLabelReader(OrdinalsReader ordsReader, LeafReaderContext readerContext)
-        throws IOException {
-      ordinalsSegmentReader = ordsReader.getReader(readerContext);
+    /** Construct from a specified {@link SortedNumericDocValues} field. */
+    public FacetLabelReader(SortedNumericDocValues ordinalValues) {
+      this.ordinalValues = ordinalValues;
     }
 
     /**
@@ -108,20 +114,28 @@ public class TaxonomyFacetLabels {
           throw new IllegalArgumentException(
               "docs out of order: previous docId=" + currentDocId + " current docId=" + docId);
         }
-        ordinalsSegmentReader.get(docId, decodedOrds);
+
         currentDocId = docId;
-        currentPos = decodedOrds.offset;
+
+        currentDocHasValues = ordinalValues.advanceExact(docId);
+        if (currentDocHasValues) {
+          currentDocOrdinalCount = ordinalValues.docValueCount();
+          currentPos = 0;
+        }
       }
 
-      int endPos = decodedOrds.offset + decodedOrds.length;
-      assert currentPos <= endPos;
+      if (currentDocHasValues == false) {
+        return null;
+      }
 
-      if (currentPos == endPos) {
-        // no more FacetLabels
+      assert currentPos <= currentDocOrdinalCount;
+      if (currentPos == currentDocOrdinalCount) {
         return null;
       }
 
-      int ord = decodedOrds.ints[currentPos++];
+      int ord = (int) ordinalValues.nextValue();
+      currentPos++;
+
       return taxoReader.getPath(ord);
     }
 
@@ -168,24 +182,36 @@ public class TaxonomyFacetLabels {
           throw new IllegalArgumentException(
               "docs out of order: previous docId=" + currentDocId + " current docId=" + docId);
         }
-        ordinalsSegmentReader.get(docId, decodedOrds);
-        currentPos = decodedOrds.offset;
         currentDocId = docId;
+
+        currentDocHasValues = ordinalValues.advanceExact(docId);
+        if (currentDocHasValues) {
+          currentDocOrdinalCount = ordinalValues.docValueCount();
+          currentPos = 0;
+        }
+      }
+
+      if (currentDocHasValues == false) {
+        return null;
+      }
+
+      assert currentPos <= currentDocOrdinalCount;
+      if (currentPos == currentDocOrdinalCount) {
+        return null;
       }
 
       if (parents == null) {
         parents = taxoReader.getParallelTaxonomyArrays().parents();
       }
 
-      int endPos = decodedOrds.offset + decodedOrds.length;
-      assert currentPos <= endPos;
-
-      for (; currentPos < endPos; ) {
-        int ord = decodedOrds.ints[currentPos++];
+      do {
+        int ord = (int) ordinalValues.nextValue();
+        currentPos++;
         if (isDescendant(ord, parentOrd) == true) {
           return taxoReader.getPath(ord);
         }
-      }
+      } while (currentPos < currentDocOrdinalCount);
+
       return null;
     }
   }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumFloatAssociations.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumFloatAssociations.java
index 5773d2e..bc6fe3a 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumFloatAssociations.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumFloatAssociations.java
@@ -60,9 +60,6 @@ public class TaxonomyFacetSumFloatAssociations extends FloatTaxonomyFacets {
 
       int doc;
       while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-        // System.out.println("  doc=" + doc);
-        // TODO: use OrdinalsReader?  we'd need to add a
-        // BytesRef getAssociation()?
         if (dv.docID() < doc) {
           dv.advance(doc);
         }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumIntAssociations.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumIntAssociations.java
index 96c23a9..4827779 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumIntAssociations.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumIntAssociations.java
@@ -60,9 +60,6 @@ public class TaxonomyFacetSumIntAssociations extends IntTaxonomyFacets {
 
       int doc;
       while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-        // System.out.println("  doc=" + doc);
-        // TODO: use OrdinalsReader?  we'd need to add a
-        // BytesRef getAssociation()?
         if (dv.docID() < doc) {
           dv.advance(doc);
         }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
index 1197da5..976eb9f 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
@@ -21,10 +21,12 @@ import java.util.List;
 import org.apache.lucene.facet.FacetsCollector;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.search.ConjunctionUtils;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.DoubleValues;
 import org.apache.lucene.search.DoubleValuesSource;
-import org.apache.lucene.util.IntsRef;
 
 /**
  * Aggregates sum of values from {@link DoubleValues#doubleValue()}, for each facet label.
@@ -32,12 +34,10 @@ import org.apache.lucene.util.IntsRef;
  * @lucene.experimental
  */
 public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
-  private final OrdinalsReader ordinalsReader;
 
   /**
    * Aggreggates double facet values from the provided {@link DoubleValuesSource}, pulling ordinals
-   * using {@link DocValuesOrdinalsReader} against the default indexed facet field {@link
-   * FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
+   * from the default indexed facet field {@link FacetsConfig#DEFAULT_INDEX_FIELD_NAME}.
    */
   public TaxonomyFacetSumValueSource(
       TaxonomyReader taxoReader,
@@ -45,28 +45,22 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
       FacetsCollector fc,
       DoubleValuesSource valueSource)
       throws IOException {
-    this(
-        new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME),
-        taxoReader,
-        config,
-        fc,
-        valueSource);
+    this(FacetsConfig.DEFAULT_INDEX_FIELD_NAME, taxoReader, config, fc, valueSource);
   }
 
   /**
-   * Aggreggates float facet values from the provided {@link DoubleValuesSource}, and pulls ordinals
-   * from the provided {@link OrdinalsReader}.
+   * Aggreggates double facet values from the provided {@link DoubleValuesSource}, pulling ordinals
+   * from the specified indexed facet field.
    */
   public TaxonomyFacetSumValueSource(
-      OrdinalsReader ordinalsReader,
+      String indexField,
       TaxonomyReader taxoReader,
       FacetsConfig config,
       FacetsCollector fc,
-      DoubleValuesSource vs)
+      DoubleValuesSource valueSource)
       throws IOException {
-    super(ordinalsReader.getIndexFieldName(), taxoReader, config);
-    this.ordinalsReader = ordinalsReader;
-    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), vs);
+    super(indexField, taxoReader, config);
+    sumValues(fc.getMatchingDocs(), fc.getKeepScores(), valueSource);
   }
 
   private static DoubleValues scores(MatchingDocs hits) {
@@ -91,20 +85,20 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
       List<MatchingDocs> matchingDocs, boolean keepScores, DoubleValuesSource valueSource)
       throws IOException {
 
-    IntsRef scratch = new IntsRef();
     for (MatchingDocs hits : matchingDocs) {
-      OrdinalsReader.OrdinalsSegmentReader ords = ordinalsReader.getReader(hits.context);
+      SortedNumericDocValues ordinalValues =
+          DocValues.getSortedNumeric(hits.context.reader(), indexFieldName);
       DoubleValues scores = keepScores ? scores(hits) : null;
       DoubleValues functionValues = valueSource.getValues(hits.context, scores);
-      DocIdSetIterator docs = hits.bits.iterator();
+      DocIdSetIterator it =
+          ConjunctionUtils.intersectIterators(List.of(hits.bits.iterator(), ordinalValues));
 
-      int doc;
-      while ((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-        ords.get(doc, scratch);
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
         if (functionValues.advanceExact(doc)) {
           float value = (float) functionValues.doubleValue();
-          for (int i = 0; i < scratch.length; i++) {
-            values[scratch.ints[i]] += value;
+          int ordinalCount = ordinalValues.docValueCount();
+          for (int i = 0; i < ordinalCount; i++) {
+            values[(int) ordinalValues.nextValue()] += value;
           }
         }
       }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
index 123fa8e..e841d48 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
@@ -153,9 +153,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
       indexEpoch = 1;
     } else {
       String epochStr = null;
-
       SegmentInfos infos = SegmentInfos.readLatestCommit(dir);
-
       Map<String, String> commitData = infos.getUserData();
       if (commitData != null) {
         epochStr = commitData.get(INDEX_EPOCH);
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java b/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java
index 246709e..2f29763 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java
@@ -25,12 +25,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
-import org.apache.lucene.facet.taxonomy.CachedOrdinalsReader;
-import org.apache.lucene.facet.taxonomy.DocValuesOrdinalsReader;
 import org.apache.lucene.facet.taxonomy.FacetLabel;
 import org.apache.lucene.facet.taxonomy.FastTaxonomyFacetCounts;
-import org.apache.lucene.facet.taxonomy.OrdinalsReader;
-import org.apache.lucene.facet.taxonomy.TaxonomyFacetCounts;
 import org.apache.lucene.facet.taxonomy.TaxonomyFacetLabels;
 import org.apache.lucene.facet.taxonomy.TaxonomyFacetLabels.FacetLabelReader;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
@@ -49,18 +45,7 @@ public abstract class FacetTestCase extends LuceneTestCase {
   public Facets getTaxonomyFacetCounts(
       TaxonomyReader taxoReader, FacetsConfig config, FacetsCollector c, String indexFieldName)
       throws IOException {
-    Facets facets;
-    if (random().nextBoolean()) {
-      facets = new FastTaxonomyFacetCounts(indexFieldName, taxoReader, config, c);
-    } else {
-      OrdinalsReader ordsReader = new DocValuesOrdinalsReader(indexFieldName);
-      if (random().nextBoolean()) {
-        ordsReader = new CachedOrdinalsReader(ordsReader);
-      }
-      facets = new TaxonomyFacetCounts(ordsReader, taxoReader, config, c);
-    }
-
-    return facets;
+    return new FastTaxonomyFacetCounts(indexFieldName, taxoReader, config, c);
   }
 
   /**
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestMultipleIndexFields.java b/lucene/facet/src/test/org/apache/lucene/facet/TestMultipleIndexFields.java
index a65b136..759b536 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/TestMultipleIndexFields.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestMultipleIndexFields.java
@@ -190,7 +190,7 @@ public class TestMultipleIndexFields extends FacetTestCase {
   private void assertOrdinalsExist(String field, IndexReader ir) throws IOException {
     for (LeafReaderContext context : ir.leaves()) {
       LeafReader r = context.reader();
-      if (r.getBinaryDocValues(field) != null) {
+      if (r.getSortedNumericDocValues(field) != null) {
         return; // not all segments must have this DocValues
       }
     }
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCachedOrdinalsReader.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCachedOrdinalsReader.java
deleted file mode 100644
index 6feafd9..0000000
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestCachedOrdinalsReader.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.facet.taxonomy;
-
-import java.io.IOException;
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.facet.FacetField;
-import org.apache.lucene.facet.FacetTestCase;
-import org.apache.lucene.facet.FacetsConfig;
-import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.IOUtils;
-import org.junit.Test;
-
-public class TestCachedOrdinalsReader extends FacetTestCase {
-
-  @Test
-  public void testWithThreads() throws Exception {
-    // LUCENE-5303: OrdinalsCache used the ThreadLocal BinaryDV instead of reader.getCoreCacheKey().
-    Directory indexDir = newDirectory();
-    Directory taxoDir = newDirectory();
-    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
-    IndexWriter writer = new IndexWriter(indexDir, conf);
-    DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
-    FacetsConfig config = new FacetsConfig();
-
-    Document doc = new Document();
-    doc.add(new FacetField("A", "1"));
-    writer.addDocument(config.build(taxoWriter, doc));
-    doc = new Document();
-    doc.add(new FacetField("A", "2"));
-    writer.addDocument(config.build(taxoWriter, doc));
-
-    final DirectoryReader reader = DirectoryReader.open(writer);
-    final CachedOrdinalsReader ordsReader =
-        new CachedOrdinalsReader(
-            new DocValuesOrdinalsReader(FacetsConfig.DEFAULT_INDEX_FIELD_NAME));
-    Thread[] threads = new Thread[3];
-    for (int i = 0; i < threads.length; i++) {
-      threads[i] =
-          new Thread("CachedOrdsThread-" + i) {
-            @Override
-            public void run() {
-              for (LeafReaderContext context : reader.leaves()) {
-                try {
-                  ordsReader.getReader(context);
-                } catch (IOException e) {
-                  throw new RuntimeException(e);
-                }
-              }
-            }
-          };
-    }
-
-    long ramBytesUsed = 0;
-    for (Thread t : threads) {
-      t.start();
-      t.join();
-      if (ramBytesUsed == 0) {
-        ramBytesUsed = ordsReader.ramBytesUsed();
-      } else {
-        assertEquals(ramBytesUsed, ordsReader.ramBytesUsed());
-      }
-    }
-
-    writer.close();
-    IOUtils.close(taxoWriter, reader, indexDir, taxoDir);
-  }
-}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetCounts.java
index 944860d..8cb23f0 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetCounts.java
@@ -235,16 +235,7 @@ public class TestTaxonomyFacetCounts extends FacetTestCase {
     searcher.search(new MatchAllDocsQuery(), c);
 
     // Uses default $facets field:
-    Facets facets;
-    if (random().nextBoolean()) {
-      facets = new FastTaxonomyFacetCounts(taxoReader, config, c);
-    } else {
-      OrdinalsReader ordsReader = new DocValuesOrdinalsReader();
-      if (random().nextBoolean()) {
-        ordsReader = new CachedOrdinalsReader(ordsReader);
-      }
-      facets = new TaxonomyFacetCounts(ordsReader, taxoReader, config, c);
-    }
+    Facets facets = new FastTaxonomyFacetCounts(taxoReader, config, c);
 
     // Ask for top 10 labels for any dims that have counts:
     List<FacetResult> results = facets.getAllDims(10);
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
index 8c3cdaf..34a4d9a 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyFacetSumValueSource.java
@@ -411,8 +411,7 @@ public class TestTaxonomyFacetSumValueSource extends FacetTestCase {
 
     Facets facets1 = getTaxonomyFacetCounts(taxoReader, config, fc);
     Facets facets2 =
-        new TaxonomyFacetSumValueSource(
-            new DocValuesOrdinalsReader("$b"), taxoReader, config, fc, DoubleValuesSource.SCORES);
+        new TaxonomyFacetSumValueSource("$b", taxoReader, config, fc, DoubleValuesSource.SCORES);
 
     assertEquals(r.maxDoc(), facets1.getTopChildren(10, "a").value.intValue());
     assertEquals(r.maxDoc(), facets2.getTopChildren(10, "b").value.doubleValue(), 1E-10);