You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by ma...@apache.org on 2023/09/26 14:59:15 UTC

[solr] branch branch_9x updated: SOLR-16989: Optimize and consolidate reuse of DocValues iterators for value retrieval (#1938)

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

magibney pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new dae78724ad8 SOLR-16989: Optimize and consolidate reuse of DocValues iterators for value retrieval (#1938)
dae78724ad8 is described below

commit dae78724ad8e1551e7d7771ab727f79cc3b596e7
Author: Michael Gibney <mi...@michaelgibney.net>
AuthorDate: Tue Sep 26 10:34:44 2023 -0400

    SOLR-16989: Optimize and consolidate reuse of DocValues iterators for value retrieval (#1938)
    
    (cherry picked from commit dafbe800c2f74086b53269f5b5a62988abadb717)
---
 solr/CHANGES.txt                                   |   1 +
 .../handler/component/RealTimeGetComponent.java    |  14 +-
 .../solr/handler/export/BoolFieldWriter.java       |   8 +-
 .../solr/handler/export/DateFieldWriter.java       |  53 +---
 .../solr/handler/export/DoubleFieldWriter.java     |  41 +--
 .../apache/solr/handler/export/ExportWriter.java   |  44 ++-
 .../solr/handler/export/FloatFieldWriter.java      |  42 +--
 .../apache/solr/handler/export/IntFieldWriter.java |  31 +-
 .../solr/handler/export/LongFieldWriter.java       |  37 +--
 .../solr/handler/export/MultiFieldWriter.java      |  64 ++---
 .../solr/handler/export/StringFieldWriter.java     |  34 +--
 .../apache/solr/search/DocValuesIteratorCache.java | 319 +++++++++++++++++++++
 .../apache/solr/search/SolrDocumentFetcher.java    |  74 ++---
 .../solr/search/TestDocValuesIteratorCache.java    | 154 ++++++++++
 14 files changed, 660 insertions(+), 256 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index ab65554711e..76f3d641727 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -89,6 +89,7 @@ Optimizations
 
 * SOLR-16265: reduce memory usage of ContentWriter based requests in Http2SolrClient (Alex Deparvu, Kevin Risden, David Smiley)
 
+* SOLR-16989: Optimize and consolidate reuse of DocValues iterators for value retrieval (Michael Gibney)
 
 Bug Fixes
 ---------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
index 7944f4840ce..a34cd99d8be 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java
@@ -81,6 +81,7 @@ import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocList;
+import org.apache.solr.search.DocValuesIteratorCache;
 import org.apache.solr.search.QueryUtils;
 import org.apache.solr.search.ReturnFields;
 import org.apache.solr.search.SolrDocumentFetcher;
@@ -238,6 +239,7 @@ public class RealTimeGetComponent extends SearchComponent {
 
       boolean opennedRealtimeSearcher = false;
       BytesRefBuilder idBytes = new BytesRefBuilder();
+      DocValuesIteratorCache reuseDvIters = null;
       for (String idStr : reqIds.allIds) {
         fieldType.readableToIndexed(idStr, idBytes);
         // if _route_ is passed, id is a child doc.  TODO remove in SOLR-15064
@@ -348,7 +350,11 @@ public class RealTimeGetComponent extends SearchComponent {
             searcherInfo.getSearcher().doc(docid, rsp.getReturnFields().getLuceneFieldNames());
         SolrDocument doc = toSolrDoc(luceneDocument, core.getLatestSchema());
         SolrDocumentFetcher docFetcher = searcherInfo.getSearcher().getDocFetcher();
-        docFetcher.decorateDocValueFields(doc, docid, docFetcher.getNonStoredDVs(true));
+        if (reuseDvIters == null) {
+          reuseDvIters = new DocValuesIteratorCache(searcherInfo.getSearcher());
+        }
+        docFetcher.decorateDocValueFields(
+            doc, docid, docFetcher.getNonStoredDVs(true), reuseDvIters);
         if (null != transformer) {
           if (null == resultContext) {
             // either first pass, or we've re-opened searcher - either way now we setContext
@@ -575,7 +581,11 @@ public class RealTimeGetComponent extends SearchComponent {
       if (!doc.containsKey(VERSION_FIELD)) {
         searcher
             .getDocFetcher()
-            .decorateDocValueFields(doc, docid, Collections.singleton(VERSION_FIELD));
+            .decorateDocValueFields(
+                doc,
+                docid,
+                Collections.singleton(VERSION_FIELD),
+                new DocValuesIteratorCache(searcher, false));
       }
 
       long docVersion = (long) doc.getFirstValue(VERSION_FIELD);
diff --git a/solr/core/src/java/org/apache/solr/handler/export/BoolFieldWriter.java b/solr/core/src/java/org/apache/solr/handler/export/BoolFieldWriter.java
index fbeccdc4c0d..cf32497b5d7 100644
--- a/solr/core/src/java/org/apache/solr/handler/export/BoolFieldWriter.java
+++ b/solr/core/src/java/org/apache/solr/handler/export/BoolFieldWriter.java
@@ -21,10 +21,14 @@ import java.io.IOException;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.search.DocValuesIteratorCache;
 
 class BoolFieldWriter extends StringFieldWriter {
-  public BoolFieldWriter(String field, FieldType fieldType) {
-    super(field, fieldType);
+  public BoolFieldWriter(
+      String field,
+      FieldType fieldType,
+      DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache) {
+    super(field, fieldType, docValuesCache);
   }
 
   @Override
diff --git a/solr/core/src/java/org/apache/solr/handler/export/DateFieldWriter.java b/solr/core/src/java/org/apache/solr/handler/export/DateFieldWriter.java
index c86f0eacac1..7e32e98d5b8 100644
--- a/solr/core/src/java/org/apache/solr/handler/export/DateFieldWriter.java
+++ b/solr/core/src/java/org/apache/solr/handler/export/DateFieldWriter.java
@@ -17,58 +17,19 @@
 
 package org.apache.solr.handler.export;
 
-import com.carrotsearch.hppc.IntObjectHashMap;
 import java.io.IOException;
 import java.util.Date;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.NumericDocValues;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.search.DocValuesIteratorCache;
 
-class DateFieldWriter extends FieldWriter {
-  private String field;
-  private IntObjectHashMap<NumericDocValues> docValuesCache = new IntObjectHashMap<>();
-
-  public DateFieldWriter(String field) {
-    this.field = field;
+class DateFieldWriter extends LongFieldWriter {
+  public DateFieldWriter(
+      String field, DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache) {
+    super(field, docValuesCache);
   }
 
   @Override
-  public boolean write(
-      SortDoc sortDoc, LeafReaderContext readerContext, MapWriter.EntryWriter ew, int fieldIndex)
-      throws IOException {
-    Long val;
-    SortValue sortValue = sortDoc.getSortValue(this.field);
-    if (sortValue != null) {
-      if (sortValue.isPresent()) {
-        val = (long) sortValue.getCurrentValue();
-      } else { // empty-value
-        return false;
-      }
-    } else {
-      // field is not part of 'sort' param, but part of 'fl' param
-      int readerOrd = readerContext.ord;
-      NumericDocValues vals = null;
-      if (docValuesCache.containsKey(readerOrd)) {
-        NumericDocValues numericDocValues = docValuesCache.get(readerOrd);
-        if (numericDocValues.docID() < sortDoc.docId) {
-          // We have not advanced beyond the current docId so we can use this docValues.
-          vals = numericDocValues;
-        }
-      }
-
-      if (vals == null) {
-        vals = DocValues.getNumeric(readerContext.reader(), this.field);
-        docValuesCache.put(readerOrd, vals);
-      }
-
-      if (vals.advance(sortDoc.docId) == sortDoc.docId) {
-        val = vals.longValue();
-      } else {
-        return false;
-      }
-    }
-    ew.put(this.field, new Date(val));
-    return true;
+  protected void doWrite(MapWriter.EntryWriter ew, long val) throws IOException {
+    ew.put(field, new Date(val));
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/export/DoubleFieldWriter.java b/solr/core/src/java/org/apache/solr/handler/export/DoubleFieldWriter.java
index 50cbdddb385..e439560894b 100644
--- a/solr/core/src/java/org/apache/solr/handler/export/DoubleFieldWriter.java
+++ b/solr/core/src/java/org/apache/solr/handler/export/DoubleFieldWriter.java
@@ -17,57 +17,46 @@
 
 package org.apache.solr.handler.export;
 
-import com.carrotsearch.hppc.IntObjectHashMap;
 import java.io.IOException;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.search.DocValuesIteratorCache;
 
 class DoubleFieldWriter extends FieldWriter {
-  private String field;
-  private IntObjectHashMap<NumericDocValues> docValuesCache = new IntObjectHashMap<>();
+  private final String field;
+  private final DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache;
 
-  public DoubleFieldWriter(String field) {
+  public DoubleFieldWriter(
+      String field, DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache) {
     this.field = field;
+    this.docValuesCache = docValuesCache;
   }
 
   @Override
   public boolean write(
       SortDoc sortDoc, LeafReaderContext readerContext, MapWriter.EntryWriter ew, int fieldIndex)
       throws IOException {
+    double val;
     SortValue sortValue = sortDoc.getSortValue(this.field);
     if (sortValue != null) {
       if (sortValue.isPresent()) {
-        double val = (double) sortValue.getCurrentValue();
-        ew.put(this.field, val);
-        return true;
+        val = (double) sortValue.getCurrentValue();
       } else { // empty-value
         return false;
       }
     } else {
       // field is not part of 'sort' param, but part of 'fl' param
-      int readerOrd = readerContext.ord;
-      NumericDocValues vals = null;
-      if (docValuesCache.containsKey(readerOrd)) {
-        NumericDocValues numericDocValues = docValuesCache.get(readerOrd);
-        if (numericDocValues.docID() < sortDoc.docId) {
-          // We have not advanced beyond the current docId so we can use this docValues.
-          vals = numericDocValues;
-        }
-      }
-
-      if (vals == null) {
-        vals = DocValues.getNumeric(readerContext.reader(), this.field);
-        docValuesCache.put(readerOrd, vals);
-      }
-      if (vals.advance(sortDoc.docId) == sortDoc.docId) {
-        long val = vals.longValue();
-        ew.put(this.field, Double.longBitsToDouble(val));
-        return true;
+      NumericDocValues vals =
+          docValuesCache.getNumericDocValues(
+              sortDoc.docId, readerContext.reader(), readerContext.ord);
+      if (vals != null) {
+        val = Double.longBitsToDouble(vals.longValue());
       } else {
         return false;
       }
     }
+    ew.put(this.field, val);
+    return true;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/export/ExportWriter.java b/solr/core/src/java/org/apache/solr/handler/export/ExportWriter.java
index 8f38c94942e..51ba5551b69 100644
--- a/solr/core/src/java/org/apache/solr/handler/export/ExportWriter.java
+++ b/solr/core/src/java/org/apache/solr/handler/export/ExportWriter.java
@@ -74,6 +74,7 @@ import org.apache.solr.schema.LongValueFieldType;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.schema.SortableTextField;
 import org.apache.solr.schema.StrField;
+import org.apache.solr.search.DocValuesIteratorCache;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.SortSpec;
 import org.apache.solr.search.SyntaxError;
@@ -99,6 +100,14 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
 
   public static final int DEFAULT_BATCH_SIZE = 30000;
   public static final int DEFAULT_QUEUE_SIZE = 150000;
+  private static final FieldWriter EMPTY_FIELD_WRITER =
+      new FieldWriter() {
+        @Override
+        public boolean write(
+            SortDoc sortDoc, LeafReaderContext readerContext, EntryWriter out, int fieldIndex) {
+          return false;
+        }
+      };
 
   private OutputStreamWriter respWriter;
   final SolrQueryRequest req;
@@ -480,6 +489,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
       throws IOException {
     IndexSchema schema = searcher.getSchema();
     FieldWriter[] writers = new FieldWriter[fields.length];
+    DocValuesIteratorCache dvIterCache = new DocValuesIteratorCache(searcher, false);
     for (int i = 0; i < fields.length; i++) {
       String field = fields[i];
       SchemaField schemaField = null;
@@ -501,47 +511,51 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable {
             schemaField + " Must have useDocValuesAsStored='true' to be used with export writer");
       }
 
-      if (fieldType instanceof IntValueFieldType) {
+      DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache = dvIterCache.getSupplier(field);
+
+      if (docValuesCache == null) {
+        writers[i] = EMPTY_FIELD_WRITER;
+      } else if (fieldType instanceof IntValueFieldType) {
         if (multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true, docValuesCache);
         } else {
-          writers[i] = new IntFieldWriter(field);
+          writers[i] = new IntFieldWriter(field, docValuesCache);
         }
       } else if (fieldType instanceof LongValueFieldType) {
         if (multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true, docValuesCache);
         } else {
-          writers[i] = new LongFieldWriter(field);
+          writers[i] = new LongFieldWriter(field, docValuesCache);
         }
       } else if (fieldType instanceof FloatValueFieldType) {
         if (multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true, docValuesCache);
         } else {
-          writers[i] = new FloatFieldWriter(field);
+          writers[i] = new FloatFieldWriter(field, docValuesCache);
         }
       } else if (fieldType instanceof DoubleValueFieldType) {
         if (multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true, docValuesCache);
         } else {
-          writers[i] = new DoubleFieldWriter(field);
+          writers[i] = new DoubleFieldWriter(field, docValuesCache);
         }
       } else if (fieldType instanceof StrField || fieldType instanceof SortableTextField) {
         if (multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, false);
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, false, docValuesCache);
         } else {
-          writers[i] = new StringFieldWriter(field, fieldType);
+          writers[i] = new StringFieldWriter(field, fieldType, docValuesCache);
         }
       } else if (fieldType instanceof DateValueFieldType) {
         if (multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, false);
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, false, docValuesCache);
         } else {
-          writers[i] = new DateFieldWriter(field);
+          writers[i] = new DateFieldWriter(field, docValuesCache);
         }
       } else if (fieldType instanceof BoolField) {
         if (multiValued) {
-          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
+          writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true, docValuesCache);
         } else {
-          writers[i] = new BoolFieldWriter(field, fieldType);
+          writers[i] = new BoolFieldWriter(field, fieldType, docValuesCache);
         }
       } else {
         throw new IOException(
diff --git a/solr/core/src/java/org/apache/solr/handler/export/FloatFieldWriter.java b/solr/core/src/java/org/apache/solr/handler/export/FloatFieldWriter.java
index 48bd9d632ed..a60c14e6b0a 100644
--- a/solr/core/src/java/org/apache/solr/handler/export/FloatFieldWriter.java
+++ b/solr/core/src/java/org/apache/solr/handler/export/FloatFieldWriter.java
@@ -17,58 +17,46 @@
 
 package org.apache.solr.handler.export;
 
-import com.carrotsearch.hppc.IntObjectHashMap;
 import java.io.IOException;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.search.DocValuesIteratorCache;
 
 class FloatFieldWriter extends FieldWriter {
-  private String field;
-  private IntObjectHashMap<NumericDocValues> docValuesCache = new IntObjectHashMap<>();
+  private final String field;
+  private final DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache;
 
-  public FloatFieldWriter(String field) {
+  public FloatFieldWriter(
+      String field, DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache) {
     this.field = field;
+    this.docValuesCache = docValuesCache;
   }
 
   @Override
   public boolean write(
       SortDoc sortDoc, LeafReaderContext readerContext, MapWriter.EntryWriter ew, int fieldIndex)
       throws IOException {
+    float val;
     SortValue sortValue = sortDoc.getSortValue(this.field);
     if (sortValue != null) {
       if (sortValue.isPresent()) {
-        float val = (float) sortValue.getCurrentValue();
-        ew.put(this.field, val);
-        return true;
+        val = (float) sortValue.getCurrentValue();
       } else { // empty-value
         return false;
       }
     } else {
       // field is not part of 'sort' param, but part of 'fl' param
-      int readerOrd = readerContext.ord;
-      NumericDocValues vals = null;
-      if (docValuesCache.containsKey(readerOrd)) {
-        NumericDocValues numericDocValues = docValuesCache.get(readerOrd);
-        if (numericDocValues.docID() < sortDoc.docId) {
-          // We have not advanced beyond the current docId so we can use this docValues.
-          vals = numericDocValues;
-        }
-      }
-
-      if (vals == null) {
-        vals = DocValues.getNumeric(readerContext.reader(), this.field);
-        docValuesCache.put(readerOrd, vals);
-      }
-
-      if (vals.advance(sortDoc.docId) == sortDoc.docId) {
-        int val = (int) vals.longValue();
-        ew.put(this.field, Float.intBitsToFloat(val));
-        return true;
+      NumericDocValues vals =
+          docValuesCache.getNumericDocValues(
+              sortDoc.docId, readerContext.reader(), readerContext.ord);
+      if (vals != null) {
+        val = Float.intBitsToFloat((int) vals.longValue());
       } else {
         return false;
       }
     }
+    ew.put(this.field, val);
+    return true;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/export/IntFieldWriter.java b/solr/core/src/java/org/apache/solr/handler/export/IntFieldWriter.java
index 1ecef85f21b..bf0396d4ab8 100644
--- a/solr/core/src/java/org/apache/solr/handler/export/IntFieldWriter.java
+++ b/solr/core/src/java/org/apache/solr/handler/export/IntFieldWriter.java
@@ -17,19 +17,20 @@
 
 package org.apache.solr.handler.export;
 
-import com.carrotsearch.hppc.IntObjectHashMap;
 import java.io.IOException;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.search.DocValuesIteratorCache;
 
 class IntFieldWriter extends FieldWriter {
-  private String field;
-  private IntObjectHashMap<NumericDocValues> docValuesCache = new IntObjectHashMap<>();
+  private final String field;
+  private final DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache;
 
-  public IntFieldWriter(String field) {
+  public IntFieldWriter(
+      String field, DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache) {
     this.field = field;
+    this.docValuesCache = docValuesCache;
   }
 
   @Override
@@ -46,22 +47,10 @@ class IntFieldWriter extends FieldWriter {
       }
     } else {
       // field is not part of 'sort' param, but part of 'fl' param
-      int readerOrd = readerContext.ord;
-      NumericDocValues vals = null;
-      if (docValuesCache.containsKey(readerOrd)) {
-        NumericDocValues numericDocValues = docValuesCache.get(readerOrd);
-        if (numericDocValues.docID() < sortDoc.docId) {
-          // We have not advanced beyond the current docId so we can use this docValues.
-          vals = numericDocValues;
-        }
-      }
-
-      if (vals == null) {
-        vals = DocValues.getNumeric(readerContext.reader(), this.field);
-        docValuesCache.put(readerOrd, vals);
-      }
-
-      if (vals.advance(sortDoc.docId) == sortDoc.docId) {
+      NumericDocValues vals =
+          docValuesCache.getNumericDocValues(
+              sortDoc.docId, readerContext.reader(), readerContext.ord);
+      if (vals != null) {
         val = (int) vals.longValue();
       } else {
         return false;
diff --git a/solr/core/src/java/org/apache/solr/handler/export/LongFieldWriter.java b/solr/core/src/java/org/apache/solr/handler/export/LongFieldWriter.java
index 9c18a72bd6d..7961549477c 100644
--- a/solr/core/src/java/org/apache/solr/handler/export/LongFieldWriter.java
+++ b/solr/core/src/java/org/apache/solr/handler/export/LongFieldWriter.java
@@ -17,20 +17,21 @@
 
 package org.apache.solr.handler.export;
 
-import com.carrotsearch.hppc.IntObjectHashMap;
 import java.io.IOException;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.search.DocValuesIteratorCache;
 
 class LongFieldWriter extends FieldWriter {
-  private String field;
+  protected final String field;
 
-  private IntObjectHashMap<NumericDocValues> docValuesCache = new IntObjectHashMap<>();
+  private final DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache;
 
-  public LongFieldWriter(String field) {
+  public LongFieldWriter(
+      String field, DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache) {
     this.field = field;
+    this.docValuesCache = docValuesCache;
   }
 
   @Override
@@ -47,28 +48,20 @@ class LongFieldWriter extends FieldWriter {
       }
     } else {
       // field is not part of 'sort' param, but part of 'fl' param
-      int readerOrd = readerContext.ord;
-      NumericDocValues vals = null;
-      if (docValuesCache.containsKey(readerOrd)) {
-        NumericDocValues numericDocValues = docValuesCache.get(readerOrd);
-        if (numericDocValues.docID() < sortDoc.docId) {
-          // We have not advanced beyond the current docId so we can use this docValues.
-          vals = numericDocValues;
-        }
-      }
-
-      if (vals == null) {
-        vals = DocValues.getNumeric(readerContext.reader(), this.field);
-        docValuesCache.put(readerOrd, vals);
-      }
-
-      if (vals.advance(sortDoc.docId) == sortDoc.docId) {
+      NumericDocValues vals =
+          docValuesCache.getNumericDocValues(
+              sortDoc.docId, readerContext.reader(), readerContext.ord);
+      if (vals != null) {
         val = vals.longValue();
       } else {
         return false;
       }
     }
-    ew.put(field, val);
+    doWrite(ew, val);
     return true;
   }
+
+  protected void doWrite(MapWriter.EntryWriter ew, long val) throws IOException {
+    ew.put(field, val);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/handler/export/MultiFieldWriter.java b/solr/core/src/java/org/apache/solr/handler/export/MultiFieldWriter.java
index 86dd8ba82e3..7f5bdee4899 100644
--- a/solr/core/src/java/org/apache/solr/handler/export/MultiFieldWriter.java
+++ b/solr/core/src/java/org/apache/solr/handler/export/MultiFieldWriter.java
@@ -17,11 +17,9 @@
 
 package org.apache.solr.handler.export;
 
-import com.carrotsearch.hppc.IntObjectHashMap;
 import java.io.IOException;
 import java.util.Date;
 import java.util.function.LongFunction;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedNumericDocValues;
@@ -33,18 +31,23 @@ import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.DocValuesIteratorCache;
 
 class MultiFieldWriter extends FieldWriter {
-  private String field;
-  private FieldType fieldType;
-  private SchemaField schemaField;
-  private boolean numeric;
-  private CharsRefBuilder cref = new CharsRefBuilder();
+  private final String field;
+  private final FieldType fieldType;
+  private final SchemaField schemaField;
+  private final boolean numeric;
+  private final CharsRefBuilder cref = new CharsRefBuilder();
   private final LongFunction<Object> bitsToValue;
-  private IntObjectHashMap<Object> docValuesCache = new IntObjectHashMap<>();
+  private final DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache;
 
   public MultiFieldWriter(
-      String field, FieldType fieldType, SchemaField schemaField, boolean numeric) {
+      String field,
+      FieldType fieldType,
+      SchemaField schemaField,
+      boolean numeric,
+      DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache) {
     this.field = field;
     this.fieldType = fieldType;
     this.schemaField = schemaField;
@@ -54,6 +57,7 @@ class MultiFieldWriter extends FieldWriter {
     } else {
       bitsToValue = null;
     }
+    this.docValuesCache = docValuesCache;
   }
 
   @Override
@@ -61,24 +65,13 @@ class MultiFieldWriter extends FieldWriter {
       SortDoc sortDoc, LeafReaderContext readerContext, MapWriter.EntryWriter out, int fieldIndex)
       throws IOException {
     if (this.fieldType.isPointField()) {
-      int readerOrd = readerContext.ord;
-      SortedNumericDocValues vals = null;
-      if (docValuesCache.containsKey(readerOrd)) {
-        SortedNumericDocValues sortedNumericDocValues =
-            (SortedNumericDocValues) docValuesCache.get(readerOrd);
-        if (sortedNumericDocValues.docID() < sortDoc.docId) {
-          // We have not advanced beyond the current docId so we can use this docValues.
-          vals = sortedNumericDocValues;
-        }
-      }
-
+      SortedNumericDocValues vals =
+          docValuesCache.getSortedNumericDocValues(
+              sortDoc.docId, readerContext.reader(), readerContext.ord);
       if (vals == null) {
-        vals = DocValues.getSortedNumeric(readerContext.reader(), this.field);
-        docValuesCache.put(readerOrd, vals);
+        return false;
       }
 
-      if (!vals.advanceExact(sortDoc.docId)) return false;
-
       final SortedNumericDocValues docVals = vals;
 
       out.put(
@@ -91,32 +84,21 @@ class MultiFieldWriter extends FieldWriter {
               });
       return true;
     } else {
-      int readerOrd = readerContext.ord;
-      SortedSetDocValues vals = null;
-      if (docValuesCache.containsKey(readerOrd)) {
-        SortedSetDocValues sortedSetDocValues = (SortedSetDocValues) docValuesCache.get(readerOrd);
-        if (sortedSetDocValues.docID() < sortDoc.docId) {
-          // We have not advanced beyond the current docId so we can use this docValues.
-          vals = sortedSetDocValues;
-        }
-      }
-
+      SortedSetDocValues vals =
+          docValuesCache.getSortedSetDocValues(
+              sortDoc.docId, readerContext.reader(), readerContext.ord);
       if (vals == null) {
-        vals = DocValues.getSortedSet(readerContext.reader(), this.field);
-        docValuesCache.put(readerOrd, vals);
+        return false;
       }
 
-      if (vals.advance(sortDoc.docId) != sortDoc.docId) return false;
-
       final SortedSetDocValues docVals = vals;
 
       out.put(
           this.field,
           (IteratorWriter)
               w -> {
-                long o;
-                while ((o = docVals.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-                  BytesRef ref = docVals.lookupOrd(o);
+                for (int i = 0, count = docVals.docValueCount(); i < count; i++) {
+                  BytesRef ref = docVals.lookupOrd(docVals.nextOrd());
                   fieldType.indexedToReadable(ref, cref);
                   IndexableField f = fieldType.createField(schemaField, cref.toString());
                   if (f == null) w.add(cref.toString());
diff --git a/solr/core/src/java/org/apache/solr/handler/export/StringFieldWriter.java b/solr/core/src/java/org/apache/solr/handler/export/StringFieldWriter.java
index 7e15704f32d..2f8d0963e3a 100644
--- a/solr/core/src/java/org/apache/solr/handler/export/StringFieldWriter.java
+++ b/solr/core/src/java/org/apache/solr/handler/export/StringFieldWriter.java
@@ -17,9 +17,7 @@
 
 package org.apache.solr.handler.export;
 
-import com.carrotsearch.hppc.IntObjectHashMap;
 import java.io.IOException;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.util.BytesRef;
@@ -28,13 +26,14 @@ import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.util.ByteArrayUtf8CharSequence;
 import org.apache.solr.common.util.JavaBinCodec;
 import org.apache.solr.schema.FieldType;
+import org.apache.solr.search.DocValuesIteratorCache;
 
 class StringFieldWriter extends FieldWriter {
-  protected String field;
-  private FieldType fieldType;
+  protected final String field;
+  private final FieldType fieldType;
   private BytesRef lastRef;
   private int lastOrd = -1;
-  private IntObjectHashMap<SortedDocValues> docValuesCache = new IntObjectHashMap<>();
+  private final DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache;
 
   protected CharsRefBuilder cref = new CharsRefBuilder();
   final ByteArrayUtf8CharSequence utf8 =
@@ -50,9 +49,13 @@ class StringFieldWriter extends FieldWriter {
         }
       };
 
-  public StringFieldWriter(String field, FieldType fieldType) {
+  public StringFieldWriter(
+      String field,
+      FieldType fieldType,
+      DocValuesIteratorCache.FieldDocValuesSupplier docValuesCache) {
     this.field = field;
     this.fieldType = fieldType;
+    this.docValuesCache = docValuesCache;
   }
 
   @Override
@@ -82,23 +85,10 @@ class StringFieldWriter extends FieldWriter {
     }
 
     if (ref == null) {
-      // Reuse the last DocValues object if possible
-      int readerOrd = readerContext.ord;
-      SortedDocValues vals = null;
-      if (docValuesCache.containsKey(readerOrd)) {
-        SortedDocValues sortedDocValues = docValuesCache.get(readerOrd);
-        if (sortedDocValues.docID() < sortDoc.docId) {
-          // We have not advanced beyond the current docId so we can use this docValues.
-          vals = sortedDocValues;
-        }
-      }
-
+      SortedDocValues vals =
+          docValuesCache.getSortedDocValues(
+              sortDoc.docId, readerContext.reader(), readerContext.ord);
       if (vals == null) {
-        vals = DocValues.getSorted(readerContext.reader(), this.field);
-        docValuesCache.put(readerOrd, vals);
-      }
-
-      if (vals.advance(sortDoc.docId) != sortDoc.docId) {
         return false;
       }
 
diff --git a/solr/core/src/java/org/apache/solr/search/DocValuesIteratorCache.java b/solr/core/src/java/org/apache/solr/search/DocValuesIteratorCache.java
new file mode 100644
index 00000000000..eba8a731881
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/DocValuesIteratorCache.java
@@ -0,0 +1,319 @@
+/*
+ * 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.solr.search;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.function.Function;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.solr.schema.SchemaField;
+
+/**
+ * A helper class for random-order value access over docValues (such as in the case of
+ * useDocValuesAsStored). This class optimizes access by reusing DocValues iterators where possible,
+ * and by narrowing the scope of DocValues per-field/per-segment (shortcircuiting attempts to
+ * `advance()` to docs that are known to have no value for a given field).
+ */
+public class DocValuesIteratorCache {
+
+  private static final EnumMap<DocValuesType, IOBiFunction<LeafReader, String, DocIdSetIterator>>
+      funcMap = new EnumMap<>(DocValuesType.class);
+
+  static {
+    funcMap.put(DocValuesType.NUMERIC, LeafReader::getNumericDocValues);
+    funcMap.put(DocValuesType.BINARY, LeafReader::getBinaryDocValues);
+    funcMap.put(
+        DocValuesType.SORTED,
+        (r, f) -> {
+          SortedDocValues dvs = r.getSortedDocValues(f);
+          return dvs == null || dvs.getValueCount() < 1 ? null : dvs;
+        });
+    funcMap.put(DocValuesType.SORTED_NUMERIC, LeafReader::getSortedNumericDocValues);
+    funcMap.put(
+        DocValuesType.SORTED_SET,
+        (r, f) -> {
+          SortedSetDocValues dvs = r.getSortedSetDocValues(f);
+          return dvs == null || dvs.getValueCount() < 1 ? null : dvs;
+        });
+  }
+
+  private static final FieldDocValuesSupplier NONE = new FieldDocValuesSupplier(null, null, 0);
+
+  private final SolrIndexSearcher searcher;
+  private final int nLeaves;
+  private final Function<String, FieldDocValuesSupplier> getSupplier;
+
+  /**
+   * Construct an instance used to optimize random-order DocValues iterator access for the specified
+   * searcher.
+   */
+  public DocValuesIteratorCache(SolrIndexSearcher searcher) {
+    this(searcher, true);
+  }
+
+  /**
+   * Construct an instance used to optimize random-order DocValues iterator access for the specified
+   * searcher.
+   *
+   * @param searcher the associated searcher
+   * @param cache if false, caching is disabled (useful mainly for single-field, single-doc access).
+   */
+  public DocValuesIteratorCache(SolrIndexSearcher searcher, boolean cache) {
+    this.searcher = searcher;
+    this.nLeaves = searcher.getTopReaderContext().leaves().size();
+    if (cache) {
+      HashMap<String, FieldDocValuesSupplier> map = new HashMap<>();
+      getSupplier = (f) -> map.computeIfAbsent(f, this::newEntry);
+    } else {
+      getSupplier = this::newEntry;
+    }
+  }
+
+  public FieldDocValuesSupplier getSupplier(String fieldName) {
+    FieldDocValuesSupplier ret = getSupplier.apply(fieldName);
+    return ret == NONE ? null : ret;
+  }
+
+  private FieldDocValuesSupplier newEntry(String fieldName) {
+    final SchemaField schemaField = searcher.getSchema().getFieldOrNull(fieldName);
+    FieldInfo fi = searcher.getFieldInfos().fieldInfo(fieldName);
+    if (schemaField == null || !schemaField.hasDocValues() || fi == null) {
+      return NONE; // Searcher doesn't have info about this field, hence ignore it.
+    }
+    final DocValuesType dvType = fi.getDocValuesType();
+    switch (dvType) {
+      case NUMERIC:
+      case BINARY:
+      case SORTED:
+      case SORTED_NUMERIC:
+      case SORTED_SET:
+        return new FieldDocValuesSupplier(schemaField, dvType, nLeaves);
+      default:
+        return NONE;
+    }
+  }
+
+  private interface IOBiFunction<T, U, R> {
+    R apply(T t, U u) throws IOException;
+  }
+
+  /**
+   * Supplies (and coordinates arbitrary-order value retrieval over) docValues iterators for a
+   * particular field, encapsulating the logic of iterator creation, reuse/caching, and advancing.
+   * Returned iterators are already positioned, and should <i>not</i> be advanced (though
+   * multi-valued iterators may consume/iterate over values/ords).
+   *
+   * <p>Instances of this class are specifically designed to support arbitrary-order value
+   * retrieval, (e.g., useDocValuesAsStored, ExportWriter) and should generally not be used for
+   * ordered retrieval (although ordered retrieval would work perfectly fine, and would add only
+   * minimal overhead).
+   */
+  public static class FieldDocValuesSupplier {
+    public final SchemaField schemaField;
+    public final DocValuesType type;
+    private final int[] minLocalIds;
+    private final int[] ceilingIds;
+    private final int[] noMatchSince;
+    private final DocIdSetIterator[] perLeaf;
+
+    private FieldDocValuesSupplier(SchemaField schemaField, DocValuesType type, int nLeaves) {
+      this.schemaField = schemaField;
+      this.type = type;
+      this.minLocalIds = new int[nLeaves];
+      Arrays.fill(minLocalIds, -1);
+      this.ceilingIds = new int[nLeaves];
+      Arrays.fill(ceilingIds, DocIdSetIterator.NO_MORE_DOCS);
+      this.noMatchSince = new int[nLeaves];
+      this.perLeaf = new DocIdSetIterator[nLeaves];
+    }
+
+    /**
+     * This method does the actual work caching iterators, determining eligibility for re-use,
+     * pulling new iterators if necessary, and determining if we have a hit for a particular doc id.
+     */
+    private DocIdSetIterator getDocValues(
+        int localId,
+        LeafReader leafReader,
+        int leafOrd,
+        boolean singleValued,
+        IOBiFunction<LeafReader, String, DocIdSetIterator> dvFunction)
+        throws IOException {
+      int min = minLocalIds[leafOrd];
+      DocIdSetIterator dv;
+      if (min == -1) {
+        // we are not yet initialized for this field/leaf.
+        dv = dvFunction.apply(leafReader, schemaField.getName());
+        if (dv == null) {
+          minLocalIds[leafOrd] = DocIdSetIterator.NO_MORE_DOCS; // cache absence of this field
+          return null;
+        }
+        // on field/leaf init, determine the min doc, so that we don't expend effort pulling
+        // new iterators for docs that fall below this floor.
+        min = dv.nextDoc();
+        minLocalIds[leafOrd] = min;
+        perLeaf[leafOrd] = dv;
+        if (localId < min) {
+          noMatchSince[leafOrd] = 0; // implicit in initial `nextDoc()` call
+          return null;
+        } else if (localId == min) {
+          noMatchSince[leafOrd] = DocIdSetIterator.NO_MORE_DOCS;
+          return dv;
+        }
+      } else if (localId < min || localId >= ceilingIds[leafOrd]) {
+        // out of range: either too low or too high
+        return null;
+      } else {
+        dv = perLeaf[leafOrd];
+        int currentDoc = dv.docID();
+        if (localId == currentDoc) {
+          if (singleValued) {
+            return dv;
+          } else if (noMatchSince[leafOrd] != DocIdSetIterator.NO_MORE_DOCS) {
+            // `noMatchSince[leafOrd] != DocIdSetIterator.NO_MORE_DOCS` means that `dv` has not
+            // been returned at its current position, and has therefore not been consumed and
+            // is thus eligible to be returned directly. (singleValued dv iterators are always
+            // eligible to be returned directly, as they have no concept of being "consumed")
+
+            // NOTE: we must reset `noMatchSince[leafOrd]` here in order to prevent returning
+            // consumed docValues; even though this actually loses us possible skipping information,
+            // it's an edge case, and allows us to use `noMatchSince[leafOrd]` as a signal of
+            // whether we have consumed multivalued docValues.
+            noMatchSince[leafOrd] = DocIdSetIterator.NO_MORE_DOCS;
+            return dv;
+          }
+        }
+        if (localId <= currentDoc) {
+          if (localId >= noMatchSince[leafOrd]) {
+            // if the requested doc falls between the last requested doc and the current
+            // position, then we know there's no match.
+            return null;
+          }
+          // we must re-init the iterator
+          dv = dvFunction.apply(leafReader, schemaField.getName());
+          perLeaf[leafOrd] = dv;
+        }
+      }
+      // NOTE: use `advance()`, not `advanceExact()`. There's no cost (in terms of re-use) to
+      // doing so, because we track `noMatchSince` in the event of a miss.
+      int found = dv.advance(localId);
+      if (found == localId) {
+        noMatchSince[leafOrd] = DocIdSetIterator.NO_MORE_DOCS;
+        return dv;
+      } else {
+        if (found == DocIdSetIterator.NO_MORE_DOCS) {
+          ceilingIds[leafOrd] = Math.min(localId, ceilingIds[leafOrd]);
+        }
+        noMatchSince[leafOrd] = localId;
+        return null;
+      }
+    }
+
+    /**
+     * Returns docValues for the specified doc id in the specified reader, if the specified doc
+     * holds docValues for this {@link FieldDocValuesSupplier} instance, otherwise returns null.
+     *
+     * <p>If a non-null value is returned, it will already positioned at the specified docId.
+     *
+     * @param localId leaf-scoped docId
+     * @param leafReader reader containing docId
+     * @param leafOrd top-level ord of the specified reader
+     */
+    public NumericDocValues getNumericDocValues(int localId, LeafReader leafReader, int leafOrd)
+        throws IOException {
+      return (NumericDocValues)
+          getDocValues(localId, leafReader, leafOrd, true, funcMap.get(DocValuesType.NUMERIC));
+    }
+
+    /**
+     * Returns docValues for the specified doc id in the specified reader, if the specified doc
+     * holds docValues for this {@link FieldDocValuesSupplier} instance, otherwise returns null.
+     *
+     * <p>If a non-null value is returned, it will already positioned at the specified docId.
+     *
+     * @param localId leaf-scoped docId
+     * @param leafReader reader containing docId
+     * @param leafOrd top-level ord of the specified reader
+     */
+    public BinaryDocValues getBinaryDocValues(int localId, LeafReader leafReader, int leafOrd)
+        throws IOException {
+      return (BinaryDocValues)
+          getDocValues(localId, leafReader, leafOrd, true, funcMap.get(DocValuesType.BINARY));
+    }
+
+    /**
+     * Returns docValues for the specified doc id in the specified reader, if the specified doc
+     * holds docValues for this {@link FieldDocValuesSupplier} instance, otherwise returns null.
+     *
+     * <p>If a non-null value is returned, it will already positioned at the specified docId.
+     *
+     * @param localId leaf-scoped docId
+     * @param leafReader reader containing docId
+     * @param leafOrd top-level ord of the specified reader
+     */
+    public SortedDocValues getSortedDocValues(int localId, LeafReader leafReader, int leafOrd)
+        throws IOException {
+      return (SortedDocValues)
+          getDocValues(localId, leafReader, leafOrd, true, funcMap.get(DocValuesType.SORTED));
+    }
+
+    /**
+     * Returns docValues for the specified doc id in the specified reader, if the specified doc
+     * holds docValues for this {@link FieldDocValuesSupplier} instance, otherwise returns null.
+     *
+     * <p>If a non-null value is returned, it will already positioned at the specified docId, and
+     * with values ({@link SortedNumericDocValues#nextValue()}) not yet consumed.
+     *
+     * @param localId leaf-scoped docId
+     * @param leafReader reader containing docId
+     * @param leafOrd top-level ord of the specified reader
+     */
+    public SortedNumericDocValues getSortedNumericDocValues(
+        int localId, LeafReader leafReader, int leafOrd) throws IOException {
+      return (SortedNumericDocValues)
+          getDocValues(
+              localId, leafReader, leafOrd, false, funcMap.get(DocValuesType.SORTED_NUMERIC));
+    }
+
+    /**
+     * Returns docValues for the specified doc id in the specified reader, if the specified doc
+     * holds docValues for this {@link FieldDocValuesSupplier} instance, otherwise returns null.
+     *
+     * <p>If a non-null value is returned, it will already positioned at the specified docId, and
+     * with ords ({@link SortedSetDocValues#nextOrd()}) not yet consumed.
+     *
+     * @param localId leaf-scoped docId
+     * @param leafReader reader containing docId
+     * @param leafOrd top-level ord of the specified reader
+     */
+    public SortedSetDocValues getSortedSetDocValues(int localId, LeafReader leafReader, int leafOrd)
+        throws IOException {
+      return (SortedSetDocValues)
+          getDocValues(localId, leafReader, leafOrd, false, funcMap.get(DocValuesType.SORTED_SET));
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java b/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
index a975337f869..e6ccb9edd18 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java
@@ -84,6 +84,8 @@ public class SolrDocumentFetcher {
 
   private final SolrIndexSearcher searcher;
 
+  private final int nLeaves;
+
   private final boolean enableLazyFieldLoading;
 
   private final SolrCache<Integer, Document> documentCache;
@@ -119,6 +121,7 @@ public class SolrDocumentFetcher {
   @SuppressWarnings({"unchecked"})
   SolrDocumentFetcher(SolrIndexSearcher searcher, SolrConfig solrConfig, boolean cachingEnabled) {
     this.searcher = searcher;
+    this.nLeaves = searcher.getTopReaderContext().leaves().size();
     this.enableLazyFieldLoading = solrConfig.enableLazyFieldLoading;
     if (cachingEnabled) {
       documentCache =
@@ -561,16 +564,23 @@ public class SolrDocumentFetcher {
    * @param fields The fields with docValues to populate the document with. DocValues fields which
    *     do not exist or not decodable will be ignored.
    */
-  public void decorateDocValueFields(SolrDocumentBase<?, ?> doc, int docid, Set<String> fields)
+  public void decorateDocValueFields(
+      SolrDocumentBase<?, ?> doc,
+      int docid,
+      Set<String> fields,
+      DocValuesIteratorCache reuseDvIters)
       throws IOException {
     final List<LeafReaderContext> leafContexts = searcher.getLeafContexts();
     final int subIndex = ReaderUtil.subIndex(docid, leafContexts);
     final int localId = docid - leafContexts.get(subIndex).docBase;
     final LeafReader leafReader = leafContexts.get(subIndex).reader();
     for (String fieldName : fields) {
-      Object fieldValue = decodeDVField(localId, leafReader, fieldName);
-      if (fieldValue != null) {
-        doc.setField(fieldName, fieldValue);
+      DocValuesIteratorCache.FieldDocValuesSupplier e = reuseDvIters.getSupplier(fieldName);
+      if (e != null) {
+        Object fieldValue = decodeDVField(localId, leafReader, subIndex, e);
+        if (fieldValue != null) {
+          doc.setField(fieldName, fieldValue);
+        }
       }
     }
   }
@@ -580,59 +590,56 @@ public class SolrDocumentFetcher {
    *
    * @return null if DV field is not exist or can not decodable
    */
-  private Object decodeDVField(int localId, LeafReader leafReader, String fieldName)
+  private Object decodeDVField(
+      int localId,
+      LeafReader leafReader,
+      int readerOrd,
+      DocValuesIteratorCache.FieldDocValuesSupplier e)
       throws IOException {
-    final SchemaField schemaField = searcher.getSchema().getFieldOrNull(fieldName);
-    FieldInfo fi = searcher.getFieldInfos().fieldInfo(fieldName);
-    if (schemaField == null || !schemaField.hasDocValues() || fi == null) {
-      return null; // Searcher doesn't have info about this field, hence ignore it.
-    }
 
-    final DocValuesType dvType = fi.getDocValuesType();
+    final DocValuesType dvType = e.type;
     switch (dvType) {
       case NUMERIC:
-        final NumericDocValues ndv = leafReader.getNumericDocValues(fieldName);
+        final NumericDocValues ndv = e.getNumericDocValues(localId, leafReader, readerOrd);
         if (ndv == null) {
           return null;
         }
-        if (!ndv.advanceExact(localId)) {
-          return null;
-        }
-        Long val = ndv.longValue();
-        return decodeNumberFromDV(schemaField, val, false);
+        long val = ndv.longValue();
+        return decodeNumberFromDV(e.schemaField, val, false);
       case BINARY:
-        BinaryDocValues bdv = leafReader.getBinaryDocValues(fieldName);
-        if (bdv != null && bdv.advanceExact(localId)) {
+        BinaryDocValues bdv = e.getBinaryDocValues(localId, leafReader, readerOrd);
+        if (bdv != null) {
           return BytesRef.deepCopyOf(bdv.binaryValue());
         }
         return null;
       case SORTED:
-        SortedDocValues sdv = leafReader.getSortedDocValues(fieldName);
-        if (sdv != null && sdv.advanceExact(localId)) {
+        SortedDocValues sdv = e.getSortedDocValues(localId, leafReader, readerOrd);
+        if (sdv != null) {
           final BytesRef bRef = sdv.lookupOrd(sdv.ordValue());
           // Special handling for Boolean fields since they're stored as 'T' and 'F'.
-          if (schemaField.getType() instanceof BoolField) {
-            return schemaField.getType().toObject(schemaField, bRef);
+          if (e.schemaField.getType() instanceof BoolField) {
+            return e.schemaField.getType().toObject(e.schemaField, bRef);
           } else {
             return bRef.utf8ToString();
           }
         }
         return null;
       case SORTED_NUMERIC:
-        final SortedNumericDocValues numericDv = leafReader.getSortedNumericDocValues(fieldName);
-        if (numericDv != null && numericDv.advance(localId) == localId) {
+        final SortedNumericDocValues numericDv =
+            e.getSortedNumericDocValues(localId, leafReader, readerOrd);
+        if (numericDv != null) {
           final int docValueCount = numericDv.docValueCount();
           final List<Object> outValues = new ArrayList<>(docValueCount);
           for (int i = 0; i < docValueCount; i++) {
             long number = numericDv.nextValue();
-            Object value = decodeNumberFromDV(schemaField, number, true);
+            Object value = decodeNumberFromDV(e.schemaField, number, true);
             // return immediately if the number is not decodable, hence won't return an empty list.
             if (value == null) {
               return null;
             }
             // normally never true but LatLonPointSpatialField uses SORTED_NUMERIC even when single
             // valued
-            else if (schemaField.multiValued() == false) {
+            else if (e.schemaField.multiValued() == false) {
               return value;
             } else {
               outValues.add(value);
@@ -643,21 +650,21 @@ public class SolrDocumentFetcher {
         }
         return null;
       case SORTED_SET:
-        final SortedSetDocValues values = leafReader.getSortedSetDocValues(fieldName);
-        if (values != null && values.getValueCount() > 0 && values.advance(localId) == localId) {
+        final SortedSetDocValues values = e.getSortedSetDocValues(localId, leafReader, readerOrd);
+        if (values != null) {
           final List<Object> outValues = new ArrayList<>();
           for (long ord = values.nextOrd();
               ord != SortedSetDocValues.NO_MORE_ORDS;
               ord = values.nextOrd()) {
             BytesRef value = values.lookupOrd(ord);
-            outValues.add(schemaField.getType().toObject(schemaField, value));
+            outValues.add(e.schemaField.getType().toObject(e.schemaField, value));
           }
           assert outValues.size() > 0;
           return outValues;
         }
         return null;
       default:
-        return null;
+        throw new IllegalStateException();
     }
   }
 
@@ -751,6 +758,8 @@ public class SolrDocumentFetcher {
 
     private final SolrReturnFields solrReturnFields;
 
+    private final DocValuesIteratorCache reuseDvIters;
+
     RetrieveFieldsOptimizer(SolrReturnFields solrReturnFields) {
       this.storedFields = calcStoredFieldsForReturn(solrReturnFields);
       this.dvFields = calcDocValueFieldsForReturn(solrReturnFields);
@@ -760,6 +769,7 @@ public class SolrDocumentFetcher {
         dvFields.addAll(storedFields);
         storedFields.clear();
       }
+      reuseDvIters = dvFields.isEmpty() ? null : new DocValuesIteratorCache(searcher);
     }
 
     /**
@@ -881,7 +891,7 @@ public class SolrDocumentFetcher {
 
         // decorate the document with non-stored docValues fields
         if (returnDVFields()) {
-          decorateDocValueFields(sdoc, luceneDocId, getDvFields());
+          decorateDocValueFields(sdoc, luceneDocId, getDvFields(), reuseDvIters);
         }
       } catch (IOException e) {
         throw new SolrException(
diff --git a/solr/core/src/test/org/apache/solr/search/TestDocValuesIteratorCache.java b/solr/core/src/test/org/apache/solr/search/TestDocValuesIteratorCache.java
new file mode 100644
index 00000000000..338be897c70
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/search/TestDocValuesIteratorCache.java
@@ -0,0 +1,154 @@
+/*
+ * 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.solr.search;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.Random;
+import java.util.Set;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.index.NoMergePolicyFactory;
+import org.apache.solr.util.EmbeddedSolrServerTestRule;
+import org.apache.solr.util.RefCounted;
+import org.apache.solr.util.SolrClientTestRule;
+import org.junit.ClassRule;
+
+public class TestDocValuesIteratorCache extends SolrTestCaseJ4 {
+
+  private static final int DOC_COUNT = 1000;
+
+  @ClassRule
+  public static final SolrClientTestRule solrClientTestRule =
+      new EmbeddedSolrServerTestRule() {
+        @Override
+        protected void before() throws Throwable {
+          // must set NoMergePolicyFactory, because OrdinalMap building depends on the predictable
+          // existence of multiple segments; if the merge policy happens to combine into a single
+          // segment, no OrdinalMap will be built, throwing off our tests
+          systemSetPropertySolrTestsMergePolicyFactory(NoMergePolicyFactory.class.getName());
+          startSolr(LuceneTestCase.createTempDir());
+        }
+
+        @Override
+        protected void after() {
+          systemClearPropertySolrTestsMergePolicyFactory();
+          super.after();
+        }
+      };
+
+  private static String fieldConfig(String fieldName, boolean multivalued) {
+    return "<field name=\""
+        + fieldName
+        + "\" type=\"string\" indexed=\"false\" stored=\"false\" docValues=\"true\" useDocValuesAsStored=\"true\" multiValued=\""
+        + multivalued
+        + "\"/>\n";
+  }
+
+  private static final String SINGLE = "single";
+  private static final String MULTI = "multi";
+
+  @SuppressWarnings("try")
+  public void test() throws Exception {
+    Path configSet = LuceneTestCase.createTempDir();
+    SolrTestCaseJ4.copyMinConf(configSet.toFile());
+    Path schemaXml = configSet.resolve("conf/schema.xml");
+    Files.writeString(
+        schemaXml,
+        Files.readString(schemaXml)
+            .replace(
+                "</schema>", fieldConfig(SINGLE, false) + fieldConfig(MULTI, true) + "</schema>"));
+
+    solrClientTestRule.newCollection().withConfigSet(configSet.toString()).create();
+
+    SolrClient client = solrClientTestRule.getSolrClient();
+
+    Random r = random();
+    String[][] expectVals = indexDocs(client, r);
+
+    try (SolrCore core =
+        ((EmbeddedSolrServer) client).getCoreContainer().getCore(DEFAULT_TEST_CORENAME)) {
+      RefCounted<SolrIndexSearcher> sref = core.getSearcher();
+      try (Closeable c = sref::decref) {
+        SolrIndexSearcher s = sref.get();
+        assertEquals(DOC_COUNT, s.maxDoc());
+        SolrDocumentFetcher docFetcher = s.getDocFetcher();
+        DocValuesIteratorCache dvIterCache = new DocValuesIteratorCache(s);
+        final Set<String> getFields = Set.of(SINGLE, MULTI);
+        final SolrDocument doc = new SolrDocument();
+        for (int i = DOC_COUNT * 10; i >= 0; i--) {
+          int checkId = r.nextInt(DOC_COUNT);
+          doc.clear();
+          docFetcher.decorateDocValueFields(doc, checkId, getFields, dvIterCache);
+          String[] expected = expectVals[checkId];
+          if (expected == null) {
+            assertTrue(doc.isEmpty());
+          } else {
+            assertEquals(2, doc.size());
+            Object singleValue = doc.getFieldValue(SINGLE);
+            Collection<Object> actualVals = doc.getFieldValues(MULTI);
+            assertEquals(expected.length, actualVals.size() + 1); // +1 for single-valued field
+            assertEquals(expected[0], singleValue);
+            int j = 1;
+            for (Object o : actualVals) {
+              assertEquals(expected[j++], o);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  private String[][] indexDocs(SolrClient client, Random r)
+      throws SolrServerException, IOException {
+    String[][] ret = new String[DOC_COUNT][];
+    int pct = r.nextInt(100);
+    for (int i = 0; i < DOC_COUNT; i++) {
+      if (r.nextInt(100) > pct) {
+        client.add(sdoc("id", Integer.toString(i)));
+      } else {
+        String str = TestUtil.randomSimpleString(r);
+        String str1 = TestUtil.randomSimpleString(r);
+        String str2 = TestUtil.randomSimpleString(r);
+        client.add(sdoc("id", Integer.toString(i), SINGLE, str, MULTI, str1, MULTI, str2));
+        int cmp = str1.compareTo(str2);
+        if (cmp == 0) {
+          ret[i] = new String[] {str, str1};
+        } else if (cmp < 0) {
+          ret[i] = new String[] {str, str1, str2};
+        } else {
+          ret[i] = new String[] {str, str2, str1};
+        }
+      }
+      if (r.nextInt(DOC_COUNT / 5) == 0) {
+        // aim for 5 segments
+        client.commit();
+      }
+    }
+    client.commit();
+    return ret;
+  }
+}