You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2014/06/16 22:36:16 UTC

svn commit: r1603000 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/java/org/apache/lucene/util/ lucene/core/src/test/org/apache/...

Author: jpountz
Date: Mon Jun 16 20:36:15 2014
New Revision: 1603000

URL: http://svn.apache.org/r1603000
Log:
LUCENE-5767: OrdinalMap optimizations.

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongValues.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestOrdinalMap.java
    lucene/dev/branches/branch_4x/lucene/facet/   (props changed)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1603000&r1=1602999&r2=1603000&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Mon Jun 16 20:36:15 2014
@@ -191,6 +191,9 @@ Optimizations
 
 * LUCENE-5751: Speed up MemoryDocValues. (Adrien Grand, Robert Muir)
 
+* LUCENE-5767: OrdinalMap optimizations, that mostly help on low cardinalities.
+  (Martijn van Groningen, Adrien Grand) 
+
 Bug fixes
 
 * LUCENE-5738: Ensure NativeFSLock prevents opening the file channel for the

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1603000&r1=1602999&r2=1603000&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Mon Jun 16 20:36:15 2014
@@ -39,6 +39,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LongBitSet;
+import org.apache.lucene.util.LongValues;
 
 /** 
  * Abstract API that consumes numeric, binary and
@@ -505,6 +506,7 @@ public abstract class DocValuesConsumer 
               int nextValue;
               AtomicReader currentReader;
               Bits currentLiveDocs;
+              LongValues currentMap;
               boolean nextIsSet;
 
               @Override
@@ -539,6 +541,7 @@ public abstract class DocValuesConsumer 
                     if (readerUpto < readers.length) {
                       currentReader = readers[readerUpto];
                       currentLiveDocs = currentReader.getLiveDocs();
+                      currentMap = map.getGlobalOrds(readerUpto);
                     }
                     docIDUpto = 0;
                     continue;
@@ -547,7 +550,7 @@ public abstract class DocValuesConsumer 
                   if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
                     nextIsSet = true;
                     int segOrd = dvs[readerUpto].getOrd(docIDUpto);
-                    nextValue = segOrd == -1 ? -1 : (int) map.getGlobalOrd(readerUpto, segOrd);
+                    nextValue = segOrd == -1 ? -1 : (int) currentMap.get(segOrd);
                     docIDUpto++;
                     return true;
                   }
@@ -707,6 +710,7 @@ public abstract class DocValuesConsumer 
               long nextValue;
               AtomicReader currentReader;
               Bits currentLiveDocs;
+              LongValues currentMap;
               boolean nextIsSet;
               long ords[] = new long[8];
               int ordUpto;
@@ -751,6 +755,7 @@ public abstract class DocValuesConsumer 
                     if (readerUpto < readers.length) {
                       currentReader = readers[readerUpto];
                       currentLiveDocs = currentReader.getLiveDocs();
+                      currentMap = map.getGlobalOrds(readerUpto);
                     }
                     docIDUpto = 0;
                     continue;
@@ -766,7 +771,7 @@ public abstract class DocValuesConsumer 
                       if (ordLength == ords.length) {
                         ords = ArrayUtil.grow(ords, ordLength+1);
                       }
-                      ords[ordLength] = map.getGlobalOrd(readerUpto, ord);
+                      ords[ordLength] = currentMap.get(ord);
                       ordLength++;
                     }
                     docIDUpto++;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1603000&r1=1602999&r2=1603000&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Mon Jun 16 20:36:15 2014
@@ -25,6 +25,7 @@ import org.apache.lucene.index.MultiTerm
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LongValues;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
 import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
@@ -373,7 +374,7 @@ public class MultiDocValues {
       return new MultiSortedSetDocValues(values, starts, mapping);
     }
   }
-  
+
   /** maps per-segment ordinals to/from global ordinal space */
   // TODO: use more efficient packed ints structures?
   // TODO: pull this out? its pretty generic (maps between N ord()-enabled TermsEnums) 
@@ -387,8 +388,10 @@ public class MultiDocValues {
     final MonotonicAppendingLongBuffer globalOrdDeltas;
     // globalOrd -> first segment container
     final AppendingPackedLongBuffer firstSegments;
-    // for every segment, segmentOrd -> (globalOrd - segmentOrd)
-    final MonotonicAppendingLongBuffer ordDeltas[];
+    // for every segment, segmentOrd -> globalOrd
+    final LongValues segmentToGlobalOrds[];
+    // ram usage
+    final long ramBytesUsed;
     
     /** 
      * Creates an ordinal map that allows mapping ords to/from a merged
@@ -398,16 +401,20 @@ public class MultiDocValues {
      *             not be dense (e.g. can be FilteredTermsEnums}.
      * @throws IOException if an I/O error occurred.
      */
-    public OrdinalMap(Object owner, TermsEnum subs[]) throws IOException {
+    public OrdinalMap(Object owner, TermsEnum subs[], float acceptableOverheadRatio) throws IOException {
       // create the ordinal mappings by pulling a termsenum over each sub's 
       // unique terms, and walking a multitermsenum over those
       this.owner = owner;
+      // even though we accept an overhead ratio, we keep these ones with COMPACT
+      // since they are only used to resolve values given a global ord, which is
+      // slow anyway
       globalOrdDeltas = new MonotonicAppendingLongBuffer(PackedInts.COMPACT);
       firstSegments = new AppendingPackedLongBuffer(PackedInts.COMPACT);
-      ordDeltas = new MonotonicAppendingLongBuffer[subs.length];
+      final MonotonicAppendingLongBuffer[] ordDeltas = new MonotonicAppendingLongBuffer[subs.length];
       for (int i = 0; i < ordDeltas.length; i++) {
-        ordDeltas[i] = new MonotonicAppendingLongBuffer();
+        ordDeltas[i] = new MonotonicAppendingLongBuffer(acceptableOverheadRatio);
       }
+      long[] ordDeltaBits = new long[subs.length];
       long segmentOrds[] = new long[subs.length];
       ReaderSlice slices[] = new ReaderSlice[subs.length];
       TermsEnumIndex indexes[] = new TermsEnumIndex[slices.length];
@@ -431,6 +438,7 @@ public class MultiDocValues {
           }
           // for each per-segment ord, map it back to the global term.
           while (segmentOrds[segmentIndex] <= segmentOrd) {
+            ordDeltaBits[segmentIndex] |= delta;
             ordDeltas[segmentIndex].add(delta);
             segmentOrds[segmentIndex]++;
           }
@@ -442,14 +450,63 @@ public class MultiDocValues {
       for (int i = 0; i < ordDeltas.length; ++i) {
         ordDeltas[i].freeze();
       }
+      // ordDeltas is typically the bottleneck, so let's see what we can do to make it faster
+      segmentToGlobalOrds = new LongValues[subs.length];
+      long ramBytesUsed = BASE_RAM_BYTES_USED + globalOrdDeltas.ramBytesUsed() + firstSegments.ramBytesUsed() + RamUsageEstimator.shallowSizeOf(segmentToGlobalOrds);
+      for (int i = 0; i < ordDeltas.length; ++i) {
+        final MonotonicAppendingLongBuffer deltas = ordDeltas[i];
+        if (ordDeltaBits[i] == 0L) {
+          // segment ords perfectly match global ordinals
+          // likely in case of low cardinalities and large segments
+          segmentToGlobalOrds[i] = LongValues.IDENTITY;
+        } else {
+          final int bitsRequired = ordDeltaBits[i] < 0 ? 64 : PackedInts.bitsRequired(ordDeltaBits[i]);
+          final long monotonicBits = deltas.ramBytesUsed() * 8;
+          final long packedBits = bitsRequired * deltas.size();
+          if (deltas.size() <= Integer.MAX_VALUE
+              && packedBits <= monotonicBits * (1 + acceptableOverheadRatio)) {
+            // monotonic compression mostly adds overhead, let's keep the mapping in plain packed ints
+            final int size = (int) deltas.size();
+            final PackedInts.Mutable newDeltas = PackedInts.getMutable(size, bitsRequired, acceptableOverheadRatio);
+            final MonotonicAppendingLongBuffer.Iterator it = deltas.iterator();
+            for (int ord = 0; ord < size; ++ord) {
+              newDeltas.set(ord, it.next());
+            }
+            assert !it.hasNext();
+            segmentToGlobalOrds[i] = new LongValues() {
+              @Override
+              public long get(long ord) {
+                return ord + newDeltas.get((int) ord);
+              }
+            };
+            ramBytesUsed += newDeltas.ramBytesUsed();
+          } else {
+            segmentToGlobalOrds[i] = new LongValues() {
+              @Override
+              public long get(long ord) {
+                return ord + deltas.get((int) ord);
+              }
+            };
+            ramBytesUsed += deltas.ramBytesUsed();
+          }
+          ramBytesUsed += RamUsageEstimator.shallowSizeOf(segmentToGlobalOrds[i]);
+        }
+      }
+      this.ramBytesUsed = ramBytesUsed;
     }
-    
+
+    /** Create an {@link OrdinalMap} with the default overhead ratio.
+     *  @see #OrdinalMap(Object, TermsEnum[], float)  */
+    public OrdinalMap(Object owner, TermsEnum subs[]) throws IOException {
+      this(owner, subs, PackedInts.DEFAULT);
+    }
+
     /** 
-     * Given a segment number and segment ordinal, returns
-     * the corresponding global ordinal.
+     * Given a segment number, return a {@link LongValues} instance that maps
+     * segment ordinals to global ordinals.
      */
-    public long getGlobalOrd(int segmentIndex, long segmentOrd) {
-      return segmentOrd + ordDeltas[segmentIndex].get(segmentOrd);
+    public LongValues getGlobalOrds(int segmentIndex) {
+      return segmentToGlobalOrds[segmentIndex];
     }
 
     /**
@@ -477,11 +534,7 @@ public class MultiDocValues {
 
     @Override
     public long ramBytesUsed() {
-      long size = BASE_RAM_BYTES_USED + globalOrdDeltas.ramBytesUsed() + firstSegments.ramBytesUsed() + RamUsageEstimator.shallowSizeOf(ordDeltas);
-      for (int i = 0; i < ordDeltas.length; i++) {
-        size += ordDeltas[i].ramBytesUsed();
-      }
-      return size;
+      return ramBytesUsed;
     }
   }
   
@@ -499,7 +552,7 @@ public class MultiDocValues {
   
     /** Creates a new MultiSortedDocValues over <code>values</code> */
     MultiSortedDocValues(SortedDocValues values[], int docStarts[], OrdinalMap mapping) throws IOException {
-      assert values.length == mapping.ordDeltas.length;
+      assert values.length == mapping.segmentToGlobalOrds.length;
       assert docStarts.length == values.length + 1;
       this.values = values;
       this.docStarts = docStarts;
@@ -510,7 +563,7 @@ public class MultiDocValues {
     public int getOrd(int docID) {
       int subIndex = ReaderUtil.subIndex(docID, docStarts);
       int segmentOrd = values[subIndex].getOrd(docID - docStarts[subIndex]);
-      return segmentOrd == -1 ? segmentOrd : (int) mapping.getGlobalOrd(subIndex, segmentOrd);
+      return segmentOrd == -1 ? segmentOrd : (int) mapping.segmentToGlobalOrds[subIndex].get(segmentOrd);
     }
  
     @Override
@@ -541,7 +594,7 @@ public class MultiDocValues {
     
     /** Creates a new MultiSortedSetDocValues over <code>values</code> */
     MultiSortedSetDocValues(SortedSetDocValues values[], int docStarts[], OrdinalMap mapping) throws IOException {
-      assert values.length == mapping.ordDeltas.length;
+      assert values.length == mapping.segmentToGlobalOrds.length;
       assert docStarts.length == values.length + 1;
       this.values = values;
       this.docStarts = docStarts;
@@ -554,7 +607,7 @@ public class MultiDocValues {
       if (segmentOrd == NO_MORE_ORDS) {
         return segmentOrd;
       } else {
-        return mapping.getGlobalOrd(currentSubIndex, segmentOrd);
+        return mapping.segmentToGlobalOrds[currentSubIndex].get(segmentOrd);
       }
     }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongValues.java?rev=1603000&r1=1602999&r2=1603000&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongValues.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/LongValues.java Mon Jun 16 20:36:15 2014
@@ -27,6 +27,16 @@ import org.apache.lucene.util.packed.Pac
  *  @lucene.internal */
 public abstract class LongValues extends NumericDocValues {
 
+  /** An instance that returns the provided value. */
+  public static final LongValues IDENTITY = new LongValues() {
+
+    @Override
+    public long get(long index) {
+      return index;
+    }
+
+  };
+
   /** Get value at <code>index</code>. */
   public abstract long get(long index);
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestOrdinalMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestOrdinalMap.java?rev=1603000&r1=1602999&r2=1603000&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestOrdinalMap.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestOrdinalMap.java Mon Jun 16 20:36:15 2014
@@ -30,6 +30,7 @@ import org.apache.lucene.index.MultiDocV
 import org.apache.lucene.index.MultiDocValues.OrdinalMap;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LongValues;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.RamUsageTester;
 import org.apache.lucene.util.TestUtil;
@@ -44,6 +45,10 @@ public class TestOrdinalMap extends Luce
       }
       return true;
     }
+
+    public boolean accept(Object o) {
+      return o != LongValues.IDENTITY;
+    }
   };
 
   public void testRamBytesUsed() throws IOException {

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java?rev=1603000&r1=1602999&r2=1603000&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java Mon Jun 16 20:36:15 2014
@@ -40,6 +40,7 @@ import org.apache.lucene.index.ReaderUti
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LongValues;
 
 /** Compute facets counts from previously
  *  indexed {@link SortedSetDocValuesFacetField},
@@ -188,7 +189,8 @@ public class SortedSetDocValuesFacetCoun
       // temp ram req'ts (sum of number of ords across all
       // segs)
       if (ordinalMap != null) {
-        int segOrd = hits.context.ord;
+        final int segOrd = hits.context.ord;
+        final LongValues ordMap = ordinalMap.getGlobalOrds(segOrd);
 
         int numSegOrds = (int) segValues.getValueCount();
 
@@ -202,7 +204,7 @@ public class SortedSetDocValuesFacetCoun
             int term = (int) segValues.nextOrd();
             while (term != SortedSetDocValues.NO_MORE_ORDS) {
               //System.out.println("      segOrd=" + segOrd + " ord=" + term + " globalOrd=" + ordinalMap.getGlobalOrd(segOrd, term));
-              counts[(int) ordinalMap.getGlobalOrd(segOrd, term)]++;
+              counts[(int) ordMap.get(term)]++;
               term = (int) segValues.nextOrd();
             }
           }
@@ -228,7 +230,7 @@ public class SortedSetDocValuesFacetCoun
             int count = segCounts[ord];
             if (count != 0) {
               //System.out.println("    migrate segOrd=" + segOrd + " ord=" + ord + " globalOrd=" + ordinalMap.getGlobalOrd(segOrd, ord));
-              counts[(int) ordinalMap.getGlobalOrd(segOrd, ord)] += count;
+              counts[(int) ordMap.get(ord)] += count;
             }
           }
         }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java?rev=1603000&r1=1602999&r2=1603000&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java Mon Jun 16 20:36:15 2014
@@ -39,6 +39,10 @@ public final class RamUsageTester {
       return true;
     }
 
+    public boolean accept(Object o) {
+      return true;
+    }
+
   };
 
   /** A filter that allows to decide on what to take into account when measuring RAM usage. */
@@ -47,6 +51,9 @@ public final class RamUsageTester {
     /** Whether the provided field should be taken into account when measuring RAM usage. */
     boolean accept(Field field);
 
+    /** Whether the provided field value should be taken into account when measuring RAM usage. */
+    boolean accept(Object o);
+
   }
 
   /**
@@ -119,7 +126,7 @@ public final class RamUsageTester {
             // Push refs for traversal later.
             for (int i = len; --i >= 0 ;) {
               final Object o = Array.get(ob, i);
-              if (o != null && !seen.contains(o)) {
+              if (o != null && !seen.contains(o) && filter.accept(o)) {
                 stack.add(o);
               }
             }            
@@ -141,7 +148,7 @@ public final class RamUsageTester {
             if (filter.accept(f)) {
               // Fast path to eliminate redundancies.
               final Object o = f.get(ob);
-              if (o != null && !seen.contains(o)) {
+              if (o != null && !seen.contains(o) && filter.accept(o)) {
                 stack.add(o);
               }
             }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java?rev=1603000&r1=1602999&r2=1603000&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java Mon Jun 16 20:36:15 2014
@@ -32,6 +32,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.LongValues;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.util.NamedList;
@@ -244,11 +245,12 @@ public class DocValuesFacets {
   
   /** accumulates per-segment single-valued facet counts, mapping to global ordinal space on-the-fly */
   static void accumSingleGeneric(int counts[], int startTermIndex, SortedDocValues si, DocIdSetIterator disi, int subIndex, OrdinalMap map) throws IOException {
+    final LongValues ordmap = map == null ? null : map.getGlobalOrds(subIndex);
     int doc;
     while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
       int term = si.getOrd(doc);
       if (map != null && term >= 0) {
-        term = (int) map.getGlobalOrd(subIndex, term);
+        term = (int) ordmap.get(term);
       }
       int arrIdx = term-startTermIndex;
       if (arrIdx>=0 && arrIdx<counts.length) counts[arrIdx]++;
@@ -290,6 +292,7 @@ public class DocValuesFacets {
     
   /** accumulates per-segment multi-valued facet counts, mapping to global ordinal space on-the-fly */
   static void accumMultiGeneric(int counts[], int startTermIndex, SortedSetDocValues si, DocIdSetIterator disi, int subIndex, OrdinalMap map) throws IOException {
+    final LongValues ordMap = map == null ? null : map.getGlobalOrds(subIndex);
     int doc;
     while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
       si.setDocument(doc);
@@ -304,7 +307,7 @@ public class DocValuesFacets {
       
       do {
         if (map != null) {
-          term = (int) map.getGlobalOrd(subIndex, term);
+          term = (int) ordMap.get(term);
         }
         int arrIdx = term-startTermIndex;
         if (arrIdx>=0 && arrIdx<counts.length) counts[arrIdx]++;
@@ -343,6 +346,7 @@ public class DocValuesFacets {
   
   /** folds counts in segment ordinal space (segCounts) into global ordinal space (counts) */
   static void migrateGlobal(int counts[], int segCounts[], int subIndex, OrdinalMap map) {
+    final LongValues ordMap = map.getGlobalOrds(subIndex);
     // missing count
     counts[0] += segCounts[0];
     
@@ -350,7 +354,7 @@ public class DocValuesFacets {
     for (int ord = 1; ord < segCounts.length; ord++) {
       int count = segCounts[ord];
       if (count != 0) {
-        counts[1+(int) map.getGlobalOrd(subIndex, ord-1)] += count;
+        counts[1+(int) ordMap.get(ord-1)] += count;
       }
     }
   }