You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/02/07 21:48:28 UTC

svn commit: r1443717 [4/14] - in /lucene/dev/trunk: ./ dev-tools/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/icu/src/java/org/apache/lucene/collation/ lucene/analysis/icu/src/test/org/apache/lucene/collation/ lucene/backwards/ luc...

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Thu Feb  7 20:48:21 2013
@@ -20,20 +20,20 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.WeakHashMap;
 
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocTermOrds;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.OrdTermState;
+import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReader;
-import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.ArrayUtil;
@@ -57,6 +57,7 @@ class FieldCacheImpl implements FieldCac
   FieldCacheImpl() {
     init();
   }
+
   private synchronized void init() {
     caches = new HashMap<Class<?>,Cache>(9);
     caches.put(Byte.TYPE, new ByteCache(this));
@@ -65,8 +66,8 @@ class FieldCacheImpl implements FieldCac
     caches.put(Float.TYPE, new FloatCache(this));
     caches.put(Long.TYPE, new LongCache(this));
     caches.put(Double.TYPE, new DoubleCache(this));
-    caches.put(DocTerms.class, new DocTermsCache(this));
-    caches.put(DocTermsIndex.class, new DocTermsIndexCache(this));
+    caches.put(BinaryDocValues.class, new BinaryDocValuesCache(this));
+    caches.put(SortedDocValues.class, new SortedDocValuesCache(this));
     caches.put(DocTermOrds.class, new DocTermOrdsCache(this));
     caches.put(DocsWithFieldCache.class, new DocsWithFieldCache(this));
   }
@@ -82,7 +83,7 @@ class FieldCacheImpl implements FieldCac
       c.purge(r);
     }
   }
-  
+
   @Override
   public synchronized CacheEntry[] getCacheEntries() {
     List<CacheEntry> result = new ArrayList<CacheEntry>(17);
@@ -90,57 +91,22 @@ class FieldCacheImpl implements FieldCac
       final Cache cache = cacheEntry.getValue();
       final Class<?> cacheType = cacheEntry.getKey();
       synchronized(cache.readerCache) {
-        for (final Map.Entry<Object,Map<Entry, Object>> readerCacheEntry : cache.readerCache.entrySet()) {
+        for (final Map.Entry<Object,Map<CacheKey, Object>> readerCacheEntry : cache.readerCache.entrySet()) {
           final Object readerKey = readerCacheEntry.getKey();
           if (readerKey == null) continue;
-          final Map<Entry, Object> innerCache = readerCacheEntry.getValue();
-          for (final Map.Entry<Entry, Object> mapEntry : innerCache.entrySet()) {
-            Entry entry = mapEntry.getKey();
-            result.add(new CacheEntryImpl(readerKey, entry.field,
-                                          cacheType, entry.custom,
-                                          mapEntry.getValue()));
+          final Map<CacheKey, Object> innerCache = readerCacheEntry.getValue();
+          for (final Map.Entry<CacheKey, Object> mapEntry : innerCache.entrySet()) {
+            CacheKey entry = mapEntry.getKey();
+            result.add(new CacheEntry(readerKey, entry.field,
+                                      cacheType, entry.custom,
+                                      mapEntry.getValue()));
           }
         }
       }
     }
     return result.toArray(new CacheEntry[result.size()]);
   }
-  
-  private static final class CacheEntryImpl extends CacheEntry {
-    private final Object readerKey;
-    private final String fieldName;
-    private final Class<?> cacheType;
-    private final Object custom;
-    private final Object value;
-    CacheEntryImpl(Object readerKey, String fieldName,
-                   Class<?> cacheType,
-                   Object custom,
-                   Object value) {
-        this.readerKey = readerKey;
-        this.fieldName = fieldName;
-        this.cacheType = cacheType;
-        this.custom = custom;
-        this.value = value;
-
-        // :HACK: for testing.
-//         if (null != locale || SortField.CUSTOM != sortFieldType) {
-//           throw new RuntimeException("Locale/sortFieldType: " + this);
-//         }
 
-    }
-    @Override
-    public Object getReaderKey() { return readerKey; }
-    @Override
-    public String getFieldName() { return fieldName; }
-    @Override
-    public Class<?> getCacheType() { return cacheType; }
-    @Override
-    public Object getCustom() { return custom; }
-    @Override
-    public Object getValue() { return value; }
-  }
-
-  
   // per-segment fieldcaches don't purge until the shared core closes.
   final SegmentReader.CoreClosedListener purgeCore = new SegmentReader.CoreClosedListener() {
     @Override
@@ -183,9 +149,9 @@ class FieldCacheImpl implements FieldCac
 
     final FieldCacheImpl wrapper;
 
-    final Map<Object,Map<Entry,Object>> readerCache = new WeakHashMap<Object,Map<Entry,Object>>();
+    final Map<Object,Map<CacheKey,Object>> readerCache = new WeakHashMap<Object,Map<CacheKey,Object>>();
     
-    protected abstract Object createValue(AtomicReader reader, Entry key, boolean setDocsWithField)
+    protected abstract Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField)
         throws IOException;
 
     /** Remove this reader from the cache, if present. */
@@ -198,13 +164,13 @@ class FieldCacheImpl implements FieldCac
 
     /** Sets the key to the value for the provided reader;
      *  if the key is already set then this doesn't change it. */
-    public void put(AtomicReader reader, Entry key, Object value) {
+    public void put(AtomicReader reader, CacheKey key, Object value) {
       final Object readerKey = reader.getCoreCacheKey();
       synchronized (readerCache) {
-        Map<Entry,Object> innerCache = readerCache.get(readerKey);
+        Map<CacheKey,Object> innerCache = readerCache.get(readerKey);
         if (innerCache == null) {
           // First time this reader is using FieldCache
-          innerCache = new HashMap<Entry,Object>();
+          innerCache = new HashMap<CacheKey,Object>();
           readerCache.put(readerKey, innerCache);
           wrapper.initReader(reader);
         }
@@ -217,15 +183,15 @@ class FieldCacheImpl implements FieldCac
       }
     }
 
-    public Object get(AtomicReader reader, Entry key, boolean setDocsWithField) throws IOException {
-      Map<Entry,Object> innerCache;
+    public Object get(AtomicReader reader, CacheKey key, boolean setDocsWithField) throws IOException {
+      Map<CacheKey,Object> innerCache;
       Object value;
       final Object readerKey = reader.getCoreCacheKey();
       synchronized (readerCache) {
         innerCache = readerCache.get(readerKey);
         if (innerCache == null) {
           // First time this reader is using FieldCache
-          innerCache = new HashMap<Entry,Object>();
+          innerCache = new HashMap<CacheKey,Object>();
           readerCache.put(readerKey, innerCache);
           wrapper.initReader(reader);
           value = null;
@@ -281,12 +247,12 @@ class FieldCacheImpl implements FieldCac
   }
 
   /** Expert: Every composite-key in the internal cache is of this type. */
-  static class Entry {
+  static class CacheKey {
     final String field;        // which Field
     final Object custom;       // which custom comparator or parser
 
     /** Creates one of these objects for a custom comparator/parser. */
-    Entry (String field, Object custom) {
+    CacheKey(String field, Object custom) {
       this.field = field;
       this.custom = custom;
     }
@@ -294,8 +260,8 @@ class FieldCacheImpl implements FieldCac
     /** Two of these are equal iff they reference the same field and type. */
     @Override
     public boolean equals (Object o) {
-      if (o instanceof Entry) {
-        Entry other = (Entry) o;
+      if (o instanceof CacheKey) {
+        CacheKey other = (CacheKey) o;
         if (other.field.equals(field)) {
           if (other.custom == null) {
             if (custom == null) return true;
@@ -314,89 +280,199 @@ class FieldCacheImpl implements FieldCac
     }
   }
 
-  // inherit javadocs
-  @Override
-  public byte[] getBytes (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
-    return getBytes(reader, field, null, setDocsWithField);
-  }
+  private static abstract class Uninvert {
 
-  // inherit javadocs
-  @Override
-  public byte[] getBytes(AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField)
-      throws IOException {
-    return (byte[]) caches.get(Byte.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
-  }
+    public Bits docsWithField;
 
-  static final class ByteCache extends Cache {
-    ByteCache(FieldCacheImpl wrapper) {
-      super(wrapper);
-    }
-    @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
-        throws IOException {
-      String field = entryKey.field;
-      ByteParser parser = (ByteParser) entryKey.custom;
-      if (parser == null) {
-        return wrapper.getBytes(reader, field, FieldCache.DEFAULT_BYTE_PARSER, setDocsWithField);
-      }
+    public void uninvert(AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
       final int maxDoc = reader.maxDoc();
-      final byte[] retArray = new byte[maxDoc];
       Terms terms = reader.terms(field);
-      FixedBitSet docsWithField = null;
       if (terms != null) {
         if (setDocsWithField) {
           final int termsDocCount = terms.getDocCount();
           assert termsDocCount <= maxDoc;
           if (termsDocCount == maxDoc) {
             // Fast case: all docs have this field:
-            wrapper.setDocsWithField(reader, field, new Bits.MatchAllBits(maxDoc));
+            docsWithField = new Bits.MatchAllBits(maxDoc);
             setDocsWithField = false;
           }
         }
-        final TermsEnum termsEnum = parser.termsEnum(terms);
-        assert termsEnum != null : "TermsEnum must not be null";
+
+        final TermsEnum termsEnum = termsEnum(terms);
+
         DocsEnum docs = null;
+        FixedBitSet docsWithField = null;
         while(true) {
           final BytesRef term = termsEnum.next();
           if (term == null) {
             break;
           }
-          final byte termval = parser.parseByte(term);
+          visitTerm(term);
           docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
           while (true) {
             final int docID = docs.nextDoc();
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
             }
-            retArray[docID] = termval;
+            visitDoc(docID);
             if (setDocsWithField) {
               if (docsWithField == null) {
                 // Lazy init
-                docsWithField = new FixedBitSet(maxDoc);
+                this.docsWithField = docsWithField = new FixedBitSet(maxDoc);
               }
               docsWithField.set(docID);
             }
           }
         }
       }
+    }
+
+    protected abstract TermsEnum termsEnum(Terms terms) throws IOException;
+    protected abstract void visitTerm(BytesRef term);
+    protected abstract void visitDoc(int docID);
+  }
+
+  // null Bits means no docs matched
+  void setDocsWithField(AtomicReader reader, String field, Bits docsWithField) {
+    final int maxDoc = reader.maxDoc();
+    final Bits bits;
+    if (docsWithField == null) {
+      bits = new Bits.MatchNoBits(maxDoc);
+    } else if (docsWithField instanceof FixedBitSet) {
+      final int numSet = ((FixedBitSet) docsWithField).cardinality();
+      if (numSet >= maxDoc) {
+        // The cardinality of the BitSet is maxDoc if all documents have a value.
+        assert numSet == maxDoc;
+        bits = new Bits.MatchAllBits(maxDoc);
+      } else {
+        bits = docsWithField;
+      }
+    } else {
+      bits = docsWithField;
+    }
+    caches.get(DocsWithFieldCache.class).put(reader, new CacheKey(field, null), bits);
+  }
+  
+  // inherit javadocs
+  public Bytes getBytes (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
+    return getBytes(reader, field, null, setDocsWithField);
+  }
+
+  // inherit javadocs
+  public Bytes getBytes(AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField)
+      throws IOException {
+    final NumericDocValues valuesIn = reader.getNumericDocValues(field);
+    if (valuesIn != null) {
+      // Not cached here by FieldCacheImpl (cached instead
+      // per-thread by SegmentReader):
+      return new Bytes() {
+        @Override
+        public byte get(int docID) {
+          return (byte) valuesIn.get(docID);
+        }
+      };
+    } else {
+      return (Bytes) caches.get(Byte.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
+    }
+  }
+
+  static class BytesFromArray extends Bytes {
+    private final byte[] values;
+
+    public BytesFromArray(byte[] values) {
+      this.values = values;
+    }
+    
+    @Override
+    public byte get(int docID) {
+      return values[docID];
+    }
+  }
+
+  static final class ByteCache extends Cache {
+    ByteCache(FieldCacheImpl wrapper) {
+      super(wrapper);
+    }
+
+    @Override
+    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField)
+        throws IOException {
+
+      int maxDoc = reader.maxDoc();
+      final byte[] values;
+      final ByteParser parser = (ByteParser) key.custom;
+      if (parser == null) {
+        // Confusing: must delegate to wrapper (vs simply
+        // setting parser = DEFAULT_SHORT_PARSER) so cache
+        // key includes DEFAULT_SHORT_PARSER:
+        return wrapper.getBytes(reader, key.field, DEFAULT_BYTE_PARSER, setDocsWithField);
+      }
+
+      values = new byte[maxDoc];
+
+      Uninvert u = new Uninvert() {
+          private byte currentValue;
+
+          @Override
+          public void visitTerm(BytesRef term) {
+            currentValue = parser.parseByte(term);
+          }
+
+          @Override
+          public void visitDoc(int docID) {
+            values[docID] = currentValue;
+          }
+
+          @Override
+          protected TermsEnum termsEnum(Terms terms) throws IOException {
+            return parser.termsEnum(terms);
+          }
+        };
+
+      u.uninvert(reader, key.field, setDocsWithField);
+
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, key.field, u.docsWithField);
       }
-      return retArray;
+
+      return new BytesFromArray(values);
     }
   }
   
   // inherit javadocs
-  @Override
-  public short[] getShorts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
+  public Shorts getShorts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getShorts(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  @Override
-  public short[] getShorts(AtomicReader reader, String field, ShortParser parser, boolean setDocsWithField)
+  public Shorts getShorts(AtomicReader reader, String field, ShortParser parser, boolean setDocsWithField)
       throws IOException {
-    return (short[]) caches.get(Short.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+    final NumericDocValues valuesIn = reader.getNumericDocValues(field);
+    if (valuesIn != null) {
+      // Not cached here by FieldCacheImpl (cached instead
+      // per-thread by SegmentReader):
+      return new Shorts() {
+        @Override
+        public short get(int docID) {
+          return (short) valuesIn.get(docID);
+        }
+      };
+    } else {
+      return (Shorts) caches.get(Short.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
+    }
+  }
+
+  static class ShortsFromArray extends Shorts {
+    private final short[] values;
+
+    public ShortsFromArray(short[] values) {
+      this.values = values;
+    }
+    
+    @Override
+    public short get(int docID) {
+      return values[docID];
+    }
   }
 
   static final class ShortCache extends Cache {
@@ -405,92 +481,94 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
-      ShortParser parser = (ShortParser) entryKey.custom;
+
+      int maxDoc = reader.maxDoc();
+      final short[] values;
+      final ShortParser parser = (ShortParser) key.custom;
       if (parser == null) {
-        return wrapper.getShorts(reader, field, FieldCache.DEFAULT_SHORT_PARSER, setDocsWithField);
+        // Confusing: must delegate to wrapper (vs simply
+        // setting parser = DEFAULT_SHORT_PARSER) so cache
+        // key includes DEFAULT_SHORT_PARSER:
+        return wrapper.getShorts(reader, key.field, DEFAULT_SHORT_PARSER, setDocsWithField);
       }
-      final int maxDoc = reader.maxDoc();
-      final short[] retArray = new short[maxDoc];
-      Terms terms = reader.terms(field);
-      FixedBitSet docsWithField = null;
-      if (terms != null) {
-        if (setDocsWithField) {
-          final int termsDocCount = terms.getDocCount();
-          assert termsDocCount <= maxDoc;
-          if (termsDocCount == maxDoc) {
-            // Fast case: all docs have this field:
-            wrapper.setDocsWithField(reader, field, new Bits.MatchAllBits(maxDoc));
-            setDocsWithField = false;
+
+      values = new short[maxDoc];
+      Uninvert u = new Uninvert() {
+          private short currentValue;
+
+          @Override
+          public void visitTerm(BytesRef term) {
+            currentValue = parser.parseShort(term);
           }
-        }
-        final TermsEnum termsEnum = parser.termsEnum(terms);
-        assert termsEnum != null : "TermsEnum must not be null";
-        DocsEnum docs = null;
-        while(true) {
-          final BytesRef term = termsEnum.next();
-          if (term == null) {
-            break;
+
+          @Override
+          public void visitDoc(int docID) {
+            values[docID] = currentValue;
           }
-          final short termval = parser.parseShort(term);
-          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
-          while (true) {
-            final int docID = docs.nextDoc();
-            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
-              break;
-            }
-            retArray[docID] = termval;
-            if (setDocsWithField) {
-              if (docsWithField == null) {
-                // Lazy init
-                docsWithField = new FixedBitSet(maxDoc);
-              }
-              docsWithField.set(docID);
-            }
+          
+          @Override
+          protected TermsEnum termsEnum(Terms terms) throws IOException {
+            return parser.termsEnum(terms);
           }
-        }
-      }
+        };
+
+      u.uninvert(reader, key.field, setDocsWithField);
+
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, key.field, u.docsWithField);
       }
-      return retArray;
+      return new ShortsFromArray(values);
     }
   }
 
-  // null Bits means no docs matched
-  void setDocsWithField(AtomicReader reader, String field, Bits docsWithField) {
-    final int maxDoc = reader.maxDoc();
-    final Bits bits;
-    if (docsWithField == null) {
-      bits = new Bits.MatchNoBits(maxDoc);
-    } else if (docsWithField instanceof FixedBitSet) {
-      final int numSet = ((FixedBitSet) docsWithField).cardinality();
-      if (numSet >= maxDoc) {
-        // The cardinality of the BitSet is maxDoc if all documents have a value.
-        assert numSet == maxDoc;
-        bits = new Bits.MatchAllBits(maxDoc);
-      } else {
-        bits = docsWithField;
-      }
-    } else {
-      bits = docsWithField;
-    }
-    caches.get(DocsWithFieldCache.class).put(reader, new Entry(field, null), bits);
-  }
-  
   // inherit javadocs
-  @Override
-  public int[] getInts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
+  public Ints getInts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getInts(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  @Override
-  public int[] getInts(AtomicReader reader, String field, IntParser parser, boolean setDocsWithField)
+  public Ints getInts(AtomicReader reader, String field, IntParser parser, boolean setDocsWithField)
       throws IOException {
-    return (int[]) caches.get(Integer.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+    final NumericDocValues valuesIn = reader.getNumericDocValues(field);
+    if (valuesIn != null) {
+      // Not cached here by FieldCacheImpl (cached instead
+      // per-thread by SegmentReader):
+      return new Ints() {
+        @Override
+        public int get(int docID) {
+          return (int) valuesIn.get(docID);
+        }
+      };
+    } else {
+      return (Ints) caches.get(Integer.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
+    }
+  }
+
+  static class IntsFromArray extends Ints {
+    private final int[] values;
+
+    public IntsFromArray(int[] values) {
+      this.values = values;
+    }
+    
+    @Override
+    public int get(int docID) {
+      return values[docID];
+    }
+  }
+
+  private static class HoldsOneThing<T> {
+    private T it;
+
+    public void set(T it) {
+      this.it = it;
+    }
+
+    public T get() {
+      return it;
+    }
   }
 
   static final class IntCache extends Cache {
@@ -499,79 +577,69 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(final AtomicReader reader, CacheKey key, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
-      IntParser parser = (IntParser) entryKey.custom;
+
+      final IntParser parser = (IntParser) key.custom;
       if (parser == null) {
+        // Confusing: must delegate to wrapper (vs simply
+        // setting parser =
+        // DEFAULT_INT_PARSER/NUMERIC_UTILS_INT_PARSER) so
+        // cache key includes
+        // DEFAULT_INT_PARSER/NUMERIC_UTILS_INT_PARSER:
         try {
-          return wrapper.getInts(reader, field, DEFAULT_INT_PARSER, setDocsWithField);
+          return wrapper.getInts(reader, key.field, DEFAULT_INT_PARSER, setDocsWithField);
         } catch (NumberFormatException ne) {
-          return wrapper.getInts(reader, field, NUMERIC_UTILS_INT_PARSER, setDocsWithField);
+          return wrapper.getInts(reader, key.field, NUMERIC_UTILS_INT_PARSER, setDocsWithField);
         }
       }
-      final int maxDoc = reader.maxDoc();
-      int[] retArray = null;
 
-      Terms terms = reader.terms(field);
-      FixedBitSet docsWithField = null;
-      if (terms != null) {
-        if (setDocsWithField) {
-          final int termsDocCount = terms.getDocCount();
-          assert termsDocCount <= maxDoc;
-          if (termsDocCount == maxDoc) {
-            // Fast case: all docs have this field:
-            wrapper.setDocsWithField(reader, field, new Bits.MatchAllBits(maxDoc));
-            setDocsWithField = false;
+      final HoldsOneThing<int[]> valuesRef = new HoldsOneThing<int[]>();
+
+      Uninvert u = new Uninvert() {
+          private int currentValue;
+          private int[] values;
+
+          @Override
+          public void visitTerm(BytesRef term) {
+            currentValue = parser.parseInt(term);
+            if (values == null) {
+              // Lazy alloc so for the numeric field case
+              // (which will hit a NumberFormatException
+              // when we first try the DEFAULT_INT_PARSER),
+              // we don't double-alloc:
+              values = new int[reader.maxDoc()];
+              valuesRef.set(values);
+            }
           }
-        }
-        final TermsEnum termsEnum = parser.termsEnum(terms);
-        assert termsEnum != null : "TermsEnum must not be null";
-        DocsEnum docs = null;
-        while(true) {
-          final BytesRef term = termsEnum.next();
-          if (term == null) {
-            break;
+
+          @Override
+          public void visitDoc(int docID) {
+            values[docID] = currentValue;
           }
-          final int termval = parser.parseInt(term);
-          if (retArray == null) {
-            // late init so numeric fields don't double allocate
-            retArray = new int[maxDoc];
+          
+          @Override
+          protected TermsEnum termsEnum(Terms terms) throws IOException {
+            return parser.termsEnum(terms);
           }
+        };
 
-          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
-          while (true) {
-            final int docID = docs.nextDoc();
-            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
-              break;
-            }
-            retArray[docID] = termval;
-            if (setDocsWithField) {
-              if (docsWithField == null) {
-                // Lazy init
-                docsWithField = new FixedBitSet(maxDoc);
-              }
-              docsWithField.set(docID);
-            }
-          }
-        }
-      }
+      u.uninvert(reader, key.field, setDocsWithField);
 
-      if (retArray == null) {
-        // no values
-        retArray = new int[maxDoc];
-      }
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, key.field, u.docsWithField);
       }
-      return retArray;
+      int[] values = valuesRef.get();
+      if (values == null) {
+        values = new int[reader.maxDoc()];
+      }
+      return new IntsFromArray(values);
     }
   }
-  
-  @Override
+
   public Bits getDocsWithField(AtomicReader reader, String field)
       throws IOException {
-    return (Bits) caches.get(DocsWithFieldCache.class).get(reader, new Entry(field, null), false);
+    return (Bits) caches.get(DocsWithFieldCache.class).get(reader, new CacheKey(field, null), false);
   }
 
   static final class DocsWithFieldCache extends Cache {
@@ -580,12 +648,23 @@ class FieldCacheImpl implements FieldCac
     }
     
     @Override
-      protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
+    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField /* ignored */)
     throws IOException {
-      final String field = entryKey.field;      
+      final String field = key.field;
+      final FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
+      final int maxDoc = reader.maxDoc();
+
+      if (fieldInfo == null) {
+        // field does not exist or has no value
+        return new Bits.MatchNoBits(maxDoc);
+      } else if (fieldInfo.hasDocValues()) {
+        // doc values are dense
+        return new Bits.MatchAllBits(maxDoc);
+      }
+
+      // Visit all docs that have terms for this field
       FixedBitSet res = null;
       Terms terms = reader.terms(field);
-      final int maxDoc = reader.maxDoc();
       if (terms != null) {
         final int termsDocCount = terms.getDocCount();
         assert termsDocCount <= maxDoc;
@@ -630,18 +709,40 @@ class FieldCacheImpl implements FieldCac
   }
 
   // inherit javadocs
-  @Override
-  public float[] getFloats (AtomicReader reader, String field, boolean setDocsWithField)
+  public Floats getFloats (AtomicReader reader, String field, boolean setDocsWithField)
     throws IOException {
     return getFloats(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  @Override
-  public float[] getFloats(AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField)
+  public Floats getFloats(AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField)
     throws IOException {
+    final NumericDocValues valuesIn = reader.getNumericDocValues(field);
+    if (valuesIn != null) {
+      // Not cached here by FieldCacheImpl (cached instead
+      // per-thread by SegmentReader):
+      return new Floats() {
+        @Override
+        public float get(int docID) {
+          return Float.intBitsToFloat((int) valuesIn.get(docID));
+        }
+      };
+    } else {
+      return (Floats) caches.get(Float.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
+    }
+  }
+
+  static class FloatsFromArray extends Floats {
+    private final float[] values;
 
-    return (float[]) caches.get(Float.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+    public FloatsFromArray(float[] values) {
+      this.values = values;
+    }
+    
+    @Override
+    public float get(int docID) {
+      return values[docID];
+    }
   }
 
   static final class FloatCache extends Cache {
@@ -650,86 +751,101 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(final AtomicReader reader, CacheKey key, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
-      FloatParser parser = (FloatParser) entryKey.custom;
+
+      final FloatParser parser = (FloatParser) key.custom;
       if (parser == null) {
+        // Confusing: must delegate to wrapper (vs simply
+        // setting parser =
+        // DEFAULT_FLOAT_PARSER/NUMERIC_UTILS_FLOAT_PARSER) so
+        // cache key includes
+        // DEFAULT_FLOAT_PARSER/NUMERIC_UTILS_FLOAT_PARSER:
         try {
-          return wrapper.getFloats(reader, field, DEFAULT_FLOAT_PARSER, setDocsWithField);
+          return wrapper.getFloats(reader, key.field, DEFAULT_FLOAT_PARSER, setDocsWithField);
         } catch (NumberFormatException ne) {
-          return wrapper.getFloats(reader, field, NUMERIC_UTILS_FLOAT_PARSER, setDocsWithField);
+          return wrapper.getFloats(reader, key.field, NUMERIC_UTILS_FLOAT_PARSER, setDocsWithField);
         }
       }
-      final int maxDoc = reader.maxDoc();
-      float[] retArray = null;
 
-      Terms terms = reader.terms(field);
-      FixedBitSet docsWithField = null;
-      if (terms != null) {
-        if (setDocsWithField) {
-          final int termsDocCount = terms.getDocCount();
-          assert termsDocCount <= maxDoc;
-          if (termsDocCount == maxDoc) {
-            // Fast case: all docs have this field:
-            wrapper.setDocsWithField(reader, field, new Bits.MatchAllBits(maxDoc));
-            setDocsWithField = false;
-          }
-        }
-        final TermsEnum termsEnum = parser.termsEnum(terms);
-        assert termsEnum != null : "TermsEnum must not be null";
-        DocsEnum docs = null;
-        while(true) {
-          final BytesRef term = termsEnum.next();
-          if (term == null) {
-            break;
+      final HoldsOneThing<float[]> valuesRef = new HoldsOneThing<float[]>();
+
+      Uninvert u = new Uninvert() {
+          private float currentValue;
+          private float[] values;
+
+          @Override
+          public void visitTerm(BytesRef term) {
+            currentValue = parser.parseFloat(term);
+            if (values == null) {
+              // Lazy alloc so for the numeric field case
+              // (which will hit a NumberFormatException
+              // when we first try the DEFAULT_INT_PARSER),
+              // we don't double-alloc:
+              values = new float[reader.maxDoc()];
+              valuesRef.set(values);
+            }
           }
-          final float termval = parser.parseFloat(term);
-          if (retArray == null) {
-            // late init so numeric fields don't double allocate
-            retArray = new float[maxDoc];
+
+          @Override
+          public void visitDoc(int docID) {
+            values[docID] = currentValue;
           }
           
-          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
-          while (true) {
-            final int docID = docs.nextDoc();
-            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
-              break;
-            }
-            retArray[docID] = termval;
-            if (setDocsWithField) {
-              if (docsWithField == null) {
-                // Lazy init
-                docsWithField = new FixedBitSet(maxDoc);
-              }
-              docsWithField.set(docID);
-            }
+          @Override
+          protected TermsEnum termsEnum(Terms terms) throws IOException {
+            return parser.termsEnum(terms);
           }
-        }
-      }
+        };
+
+      u.uninvert(reader, key.field, setDocsWithField);
 
-      if (retArray == null) {
-        // no values
-        retArray = new float[maxDoc];
-      }
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, key.field, u.docsWithField);
       }
-      return retArray;
+
+      float[] values = valuesRef.get();
+      if (values == null) {
+        values = new float[reader.maxDoc()];
+      }
+      return new FloatsFromArray(values);
     }
   }
 
-
-  @Override
-  public long[] getLongs(AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
+  // inherit javadocs
+  public Longs getLongs(AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getLongs(reader, field, null, setDocsWithField);
   }
   
   // inherit javadocs
-  @Override
-  public long[] getLongs(AtomicReader reader, String field, FieldCache.LongParser parser, boolean setDocsWithField)
+  public Longs getLongs(AtomicReader reader, String field, FieldCache.LongParser parser, boolean setDocsWithField)
       throws IOException {
-    return (long[]) caches.get(Long.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+    final NumericDocValues valuesIn = reader.getNumericDocValues(field);
+    if (valuesIn != null) {
+      // Not cached here by FieldCacheImpl (cached instead
+      // per-thread by SegmentReader):
+      return new Longs() {
+        @Override
+        public long get(int docID) {
+          return valuesIn.get(docID);
+        }
+      };
+    } else {
+      return (Longs) caches.get(Long.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
+    }
+  }
+
+  static class LongsFromArray extends Longs {
+    private final long[] values;
+
+    public LongsFromArray(long[] values) {
+      this.values = values;
+    }
+    
+    @Override
+    public long get(int docID) {
+      return values[docID];
+    }
   }
 
   static final class LongCache extends Cache {
@@ -738,87 +854,101 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(final AtomicReader reader, CacheKey key, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
-      FieldCache.LongParser parser = (FieldCache.LongParser) entryKey.custom;
+
+      final LongParser parser = (LongParser) key.custom;
       if (parser == null) {
+        // Confusing: must delegate to wrapper (vs simply
+        // setting parser =
+        // DEFAULT_LONG_PARSER/NUMERIC_UTILS_LONG_PARSER) so
+        // cache key includes
+        // DEFAULT_LONG_PARSER/NUMERIC_UTILS_LONG_PARSER:
         try {
-          return wrapper.getLongs(reader, field, DEFAULT_LONG_PARSER, setDocsWithField);
+          return wrapper.getLongs(reader, key.field, DEFAULT_LONG_PARSER, setDocsWithField);
         } catch (NumberFormatException ne) {
-          return wrapper.getLongs(reader, field, NUMERIC_UTILS_LONG_PARSER, setDocsWithField);
+          return wrapper.getLongs(reader, key.field, NUMERIC_UTILS_LONG_PARSER, setDocsWithField);
         }
       }
-      final int maxDoc = reader.maxDoc();
-      long[] retArray = null;
 
-      Terms terms = reader.terms(field);
-      FixedBitSet docsWithField = null;
-      if (terms != null) {
-        if (setDocsWithField) {
-          final int termsDocCount = terms.getDocCount();
-          assert termsDocCount <= maxDoc;
-          if (termsDocCount == maxDoc) {
-            // Fast case: all docs have this field:
-            wrapper.setDocsWithField(reader, field, new Bits.MatchAllBits(maxDoc));
-            setDocsWithField = false;
+      final HoldsOneThing<long[]> valuesRef = new HoldsOneThing<long[]>();
+
+      Uninvert u = new Uninvert() {
+          private long currentValue;
+          private long[] values;
+
+          @Override
+          public void visitTerm(BytesRef term) {
+            currentValue = parser.parseLong(term);
+            if (values == null) {
+              // Lazy alloc so for the numeric field case
+              // (which will hit a NumberFormatException
+              // when we first try the DEFAULT_INT_PARSER),
+              // we don't double-alloc:
+              values = new long[reader.maxDoc()];
+              valuesRef.set(values);
+            }
           }
-        }
-        final TermsEnum termsEnum = parser.termsEnum(terms);
-        assert termsEnum != null : "TermsEnum must not be null";
-        DocsEnum docs = null;
-        while(true) {
-          final BytesRef term = termsEnum.next();
-          if (term == null) {
-            break;
+
+          @Override
+          public void visitDoc(int docID) {
+            values[docID] = currentValue;
           }
-          final long termval = parser.parseLong(term);
-          if (retArray == null) {
-            // late init so numeric fields don't double allocate
-            retArray = new long[maxDoc];
+          
+          @Override
+          protected TermsEnum termsEnum(Terms terms) throws IOException {
+            return parser.termsEnum(terms);
           }
+        };
 
-          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
-          while (true) {
-            final int docID = docs.nextDoc();
-            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
-              break;
-            }
-            retArray[docID] = termval;
-            if (setDocsWithField) {
-              if (docsWithField == null) {
-                // Lazy init
-                docsWithField = new FixedBitSet(maxDoc);
-              }
-              docsWithField.set(docID);
-            }
-          }
-        }
-      }
+      u.uninvert(reader, key.field, setDocsWithField);
 
-      if (retArray == null) {
-        // no values
-        retArray = new long[maxDoc];
-      }
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, key.field, u.docsWithField);
+      }
+      long[] values = valuesRef.get();
+      if (values == null) {
+        values = new long[reader.maxDoc()];
       }
-      return retArray;
+      return new LongsFromArray(values);
     }
   }
 
   // inherit javadocs
-  @Override
-  public double[] getDoubles(AtomicReader reader, String field, boolean setDocsWithField)
+  public Doubles getDoubles(AtomicReader reader, String field, boolean setDocsWithField)
     throws IOException {
     return getDoubles(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  @Override
-  public double[] getDoubles(AtomicReader reader, String field, FieldCache.DoubleParser parser, boolean setDocsWithField)
+  public Doubles getDoubles(AtomicReader reader, String field, FieldCache.DoubleParser parser, boolean setDocsWithField)
       throws IOException {
-    return (double[]) caches.get(Double.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+    final NumericDocValues valuesIn = reader.getNumericDocValues(field);
+    if (valuesIn != null) {
+      // Not cached here by FieldCacheImpl (cached instead
+      // per-thread by SegmentReader):
+      return new Doubles() {
+        @Override
+        public double get(int docID) {
+          return Double.longBitsToDouble(valuesIn.get(docID));
+        }
+      };
+    } else {
+      return (Doubles) caches.get(Double.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
+    }
+  }
+
+  static class DoublesFromArray extends Doubles {
+    private final double[] values;
+
+    public DoublesFromArray(double[] values) {
+      this.values = values;
+    }
+    
+    @Override
+    public double get(int docID) {
+      return values[docID];
+    }
   }
 
   static final class DoubleCache extends Cache {
@@ -827,80 +957,73 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(final AtomicReader reader, CacheKey key, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
-      FieldCache.DoubleParser parser = (FieldCache.DoubleParser) entryKey.custom;
+
+      final DoubleParser parser = (DoubleParser) key.custom;
       if (parser == null) {
+        // Confusing: must delegate to wrapper (vs simply
+        // setting parser =
+        // DEFAULT_DOUBLE_PARSER/NUMERIC_UTILS_DOUBLE_PARSER) so
+        // cache key includes
+        // DEFAULT_DOUBLE_PARSER/NUMERIC_UTILS_DOUBLE_PARSER:
         try {
-          return wrapper.getDoubles(reader, field, DEFAULT_DOUBLE_PARSER, setDocsWithField);
+          return wrapper.getDoubles(reader, key.field, DEFAULT_DOUBLE_PARSER, setDocsWithField);
         } catch (NumberFormatException ne) {
-          return wrapper.getDoubles(reader, field, NUMERIC_UTILS_DOUBLE_PARSER, setDocsWithField);
+          return wrapper.getDoubles(reader, key.field, NUMERIC_UTILS_DOUBLE_PARSER, setDocsWithField);
         }
       }
-      final int maxDoc = reader.maxDoc();
-      double[] retArray = null;
 
-      Terms terms = reader.terms(field);
-      FixedBitSet docsWithField = null;
-      if (terms != null) {
-        if (setDocsWithField) {
-          final int termsDocCount = terms.getDocCount();
-          assert termsDocCount <= maxDoc;
-          if (termsDocCount == maxDoc) {
-            // Fast case: all docs have this field:
-            wrapper.setDocsWithField(reader, field, new Bits.MatchAllBits(maxDoc));
-            setDocsWithField = false;
+      final HoldsOneThing<double[]> valuesRef = new HoldsOneThing<double[]>();
+
+      Uninvert u = new Uninvert() {
+          private double currentValue;
+          private double[] values;
+
+          @Override
+          public void visitTerm(BytesRef term) {
+            currentValue = parser.parseDouble(term);
+            if (values == null) {
+              // Lazy alloc so for the numeric field case
+              // (which will hit a NumberFormatException
+              // when we first try the DEFAULT_INT_PARSER),
+              // we don't double-alloc:
+              values = new double[reader.maxDoc()];
+              valuesRef.set(values);
+            }
           }
-        }
-        final TermsEnum termsEnum = parser.termsEnum(terms);
-        assert termsEnum != null : "TermsEnum must not be null";
-        DocsEnum docs = null;
-        while(true) {
-          final BytesRef term = termsEnum.next();
-          if (term == null) {
-            break;
+
+          @Override
+          public void visitDoc(int docID) {
+            values[docID] = currentValue;
           }
-          final double termval = parser.parseDouble(term);
-          if (retArray == null) {
-            // late init so numeric fields don't double allocate
-            retArray = new double[maxDoc];
+          
+          @Override
+          protected TermsEnum termsEnum(Terms terms) throws IOException {
+            return parser.termsEnum(terms);
           }
+        };
+
+      u.uninvert(reader, key.field, setDocsWithField);
 
-          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
-          while (true) {
-            final int docID = docs.nextDoc();
-            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
-              break;
-            }
-            retArray[docID] = termval;
-            if (setDocsWithField) {
-              if (docsWithField == null) {
-                // Lazy init
-                docsWithField = new FixedBitSet(maxDoc);
-              }
-              docsWithField.set(docID);
-            }
-          }
-        }
-      }
-      if (retArray == null) { // no values
-        retArray = new double[maxDoc];
-      }
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, key.field, u.docsWithField);
+      }
+      double[] values = valuesRef.get();
+      if (values == null) {
+        values = new double[reader.maxDoc()];
       }
-      return retArray;
+      return new DoublesFromArray(values);
     }
   }
 
-  public static class DocTermsIndexImpl extends DocTermsIndex {
+  public static class SortedDocValuesImpl extends SortedDocValues {
     private final PagedBytes.Reader bytes;
     private final PackedInts.Reader termOrdToBytesOffset;
     private final PackedInts.Reader docToTermOrd;
     private final int numOrd;
 
-    public DocTermsIndexImpl(PagedBytes.Reader bytes, PackedInts.Reader termOrdToBytesOffset, PackedInts.Reader docToTermOrd, int numOrd) {
+    public SortedDocValuesImpl(PagedBytes.Reader bytes, PackedInts.Reader termOrdToBytesOffset, PackedInts.Reader docToTermOrd, int numOrd) {
       this.bytes = bytes;
       this.docToTermOrd = docToTermOrd;
       this.termOrdToBytesOffset = termOrdToBytesOffset;
@@ -908,189 +1031,63 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    public PackedInts.Reader getDocToOrd() {
-      return docToTermOrd;
-    }
-
-    @Override
-    public int numOrd() {
+    public int getValueCount() {
       return numOrd;
     }
 
     @Override
     public int getOrd(int docID) {
-      return (int) docToTermOrd.get(docID);
-    }
-
-    @Override
-    public int size() {
-      return docToTermOrd.size();
-    }
-
-    @Override
-    public BytesRef lookup(int ord, BytesRef ret) {
-      return bytes.fill(ret, termOrdToBytesOffset.get(ord));
+      // Subtract 1, matching the 1+ord we did when
+      // storing, so that missing values, which are 0 in the
+      // packed ints, are returned as -1 ord:
+      return (int) docToTermOrd.get(docID)-1;
     }
 
     @Override
-    public TermsEnum getTermsEnum() {
-      return this.new DocTermsIndexEnum();
-    }
-
-    class DocTermsIndexEnum extends TermsEnum {
-      int currentOrd;
-      int currentBlockNumber;
-      int end;  // end position in the current block
-      final byte[][] blocks;
-      final int[] blockEnds;
-
-      final BytesRef term = new BytesRef();
-
-      public DocTermsIndexEnum() {
-        currentOrd = 0;
-        currentBlockNumber = 0;
-        blocks = bytes.getBlocks();
-        blockEnds = bytes.getBlockEnds();
-        currentBlockNumber = bytes.fillAndGetIndex(term, termOrdToBytesOffset.get(0));
-        end = blockEnds[currentBlockNumber];
-      }
-
-      @Override
-      public SeekStatus seekCeil(BytesRef text, boolean useCache /* ignored */) throws IOException {
-        int low = 1;
-        int high = numOrd-1;
-        
-        while (low <= high) {
-          int mid = (low + high) >>> 1;
-          seekExact(mid);
-          int cmp = term.compareTo(text);
-
-          if (cmp < 0)
-            low = mid + 1;
-          else if (cmp > 0)
-            high = mid - 1;
-          else
-            return SeekStatus.FOUND; // key found
-        }
-        
-        if (low == numOrd) {
-          return SeekStatus.END;
-        } else {
-          seekExact(low);
-          return SeekStatus.NOT_FOUND;
-        }
-      }
-
-      @Override
-      public void seekExact(long ord) throws IOException {
-        assert(ord >= 0 && ord <= numOrd);
-        // TODO: if gap is small, could iterate from current position?  Or let user decide that?
-        currentBlockNumber = bytes.fillAndGetIndex(term, termOrdToBytesOffset.get((int)ord));
-        end = blockEnds[currentBlockNumber];
-        currentOrd = (int)ord;
-      }
-
-      @Override
-      public BytesRef next() throws IOException {
-        int start = term.offset + term.length;
-        if (start >= end) {
-          // switch byte blocks
-          if (currentBlockNumber +1 >= blocks.length) {
-            return null;
-          }
-          currentBlockNumber++;
-          term.bytes = blocks[currentBlockNumber];
-          end = blockEnds[currentBlockNumber];
-          start = 0;
-          if (end<=0) return null;  // special case of empty last array
-        }
-
-        currentOrd++;
-
-        byte[] block = term.bytes;
-        if ((block[start] & 128) == 0) {
-          term.length = block[start];
-          term.offset = start+1;
-        } else {
-          term.length = (((block[start] & 0x7f)) << 8) | (block[1+start] & 0xff);
-          term.offset = start+2;
-        }
-
-        return term;
-      }
-
-      @Override
-      public BytesRef term() throws IOException {
-        return term;
-      }
-
-      @Override
-      public long ord() throws IOException {
-        return currentOrd;
-      }
-
-      @Override
-      public int docFreq() {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public long totalTermFreq() {
-        return -1;
-      }
-
-      @Override
-      public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public Comparator<BytesRef> getComparator() {
-        return BytesRef.getUTF8SortedAsUnicodeComparator();
-      }
-
-      @Override
-      public void seekExact(BytesRef term, TermState state) throws IOException {
-        assert state != null && state instanceof OrdTermState;
-        this.seekExact(((OrdTermState)state).ord);
-      }
-
-      @Override
-      public TermState termState() throws IOException {
-        OrdTermState state = new OrdTermState();
-        state.ord = currentOrd;
-        return state;
+    public void lookupOrd(int ord, BytesRef ret) {
+      if (ord < 0) {
+        throw new IllegalArgumentException("ord must be >=0 (got ord=" + ord + ")");
       }
+      bytes.fill(ret, termOrdToBytesOffset.get(ord));
     }
   }
 
-  @Override
-  public DocTermsIndex getTermsIndex(AtomicReader reader, String field) throws IOException {
+  public SortedDocValues getTermsIndex(AtomicReader reader, String field) throws IOException {
     return getTermsIndex(reader, field, PackedInts.FAST);
   }
 
-  @Override
-  public DocTermsIndex getTermsIndex(AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException {
-    return (DocTermsIndex) caches.get(DocTermsIndex.class).get(reader, new Entry(field, acceptableOverheadRatio), false);
+  public SortedDocValues getTermsIndex(AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException {
+    SortedDocValues valuesIn = reader.getSortedDocValues(field);
+    if (valuesIn != null) {
+      // Not cached here by FieldCacheImpl (cached instead
+      // per-thread by SegmentReader):
+      return valuesIn;
+    } else {
+      final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
+      if (info != null && !info.isIndexed() && info.hasDocValues()) {
+        // we don't try to build a sorted instance from numeric/binary doc
+        // values because dedup can be very costly
+        throw new IllegalArgumentException("Cannot get terms index for \"" + field
+            + "\": it isn't indexed and doesn't have sorted doc values");
+      }
+      return (SortedDocValues) caches.get(SortedDocValues.class).get(reader, new CacheKey(field, acceptableOverheadRatio), false);
+    }
   }
 
-  static class DocTermsIndexCache extends Cache {
-    DocTermsIndexCache(FieldCacheImpl wrapper) {
+  static class SortedDocValuesCache extends Cache {
+    SortedDocValuesCache(FieldCacheImpl wrapper) {
       super(wrapper);
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
+    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField /* ignored */)
         throws IOException {
 
-      Terms terms = reader.terms(entryKey.field);
+      final int maxDoc = reader.maxDoc();
 
-      final float acceptableOverheadRatio = ((Float) entryKey.custom).floatValue();
+      Terms terms = reader.terms(key.field);
+
+      final float acceptableOverheadRatio = ((Float) key.custom).floatValue();
 
       final PagedBytes bytes = new PagedBytes(15);
 
@@ -1098,7 +1095,6 @@ class FieldCacheImpl implements FieldCac
       int startTermsBPV;
       int startNumUniqueTerms;
 
-      int maxDoc = reader.maxDoc();
       final int termCountHardLimit;
       if (maxDoc == Integer.MAX_VALUE) {
         termCountHardLimit = Integer.MAX_VALUE;
@@ -1106,6 +1102,7 @@ class FieldCacheImpl implements FieldCac
         termCountHardLimit = maxDoc+1;
       }
 
+      // TODO: use Uninvert?
       if (terms != null) {
         // Try for coarse estimate for number of bits; this
         // should be an underestimate most of the time, which
@@ -1137,9 +1134,9 @@ class FieldCacheImpl implements FieldCac
       GrowableWriter termOrdToBytesOffset = new GrowableWriter(startBytesBPV, 1+startNumUniqueTerms, acceptableOverheadRatio);
       final GrowableWriter docToTermOrd = new GrowableWriter(startTermsBPV, maxDoc, acceptableOverheadRatio);
 
-      // 0 is reserved for "unset"
-      bytes.copyUsingLengthPrefix(new BytesRef());
-      int termOrd = 1;
+      int termOrd = 0;
+
+      // TODO: use Uninvert?
 
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator(null);
@@ -1167,7 +1164,8 @@ class FieldCacheImpl implements FieldCac
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
             }
-            docToTermOrd.set(docID, termOrd);
+            // Store 1+ ord into packed bits
+            docToTermOrd.set(docID, 1+termOrd);
           }
           termOrd++;
         }
@@ -1178,62 +1176,72 @@ class FieldCacheImpl implements FieldCac
       }
 
       // maybe an int-only impl?
-      return new DocTermsIndexImpl(bytes.freeze(true), termOrdToBytesOffset.getMutable(), docToTermOrd.getMutable(), termOrd);
+      return new SortedDocValuesImpl(bytes.freeze(true), termOrdToBytesOffset.getMutable(), docToTermOrd.getMutable(), termOrd);
     }
   }
 
-  private static class DocTermsImpl extends DocTerms {
+  private static class BinaryDocValuesImpl extends BinaryDocValues {
     private final PagedBytes.Reader bytes;
     private final PackedInts.Reader docToOffset;
 
-    public DocTermsImpl(PagedBytes.Reader bytes, PackedInts.Reader docToOffset) {
+    public BinaryDocValuesImpl(PagedBytes.Reader bytes, PackedInts.Reader docToOffset) {
       this.bytes = bytes;
       this.docToOffset = docToOffset;
     }
 
     @Override
-    public int size() {
-      return docToOffset.size();
-    }
-
-    @Override
-    public boolean exists(int docID) {
-      return docToOffset.get(docID) == 0;
-    }
-
-    @Override
-    public BytesRef getTerm(int docID, BytesRef ret) {
+    public void get(int docID, BytesRef ret) {
       final int pointer = (int) docToOffset.get(docID);
-      return bytes.fill(ret, pointer);
-    }      
+      if (pointer == 0) {
+        ret.bytes = MISSING;
+        ret.offset = 0;
+        ret.length = 0;
+      } else {
+        bytes.fill(ret, pointer);
+      }
+    }
   }
 
   // TODO: this if DocTermsIndex was already created, we
   // should share it...
-  @Override
-  public DocTerms getTerms(AtomicReader reader, String field) throws IOException {
+  public BinaryDocValues getTerms(AtomicReader reader, String field) throws IOException {
     return getTerms(reader, field, PackedInts.FAST);
   }
 
-  @Override
-  public DocTerms getTerms(AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException {
-    return (DocTerms) caches.get(DocTerms.class).get(reader, new Entry(field, acceptableOverheadRatio), false);
+  public BinaryDocValues getTerms(AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException {
+    BinaryDocValues valuesIn = reader.getBinaryDocValues(field);
+    if (valuesIn == null) {
+      valuesIn = reader.getSortedDocValues(field);
+    }
+
+    if (valuesIn != null) {
+      // Not cached here by FieldCacheImpl (cached instead
+      // per-thread by SegmentReader):
+      return valuesIn;
+    }
+
+    return (BinaryDocValues) caches.get(BinaryDocValues.class).get(reader, new CacheKey(field, acceptableOverheadRatio), false);
   }
 
-  static final class DocTermsCache extends Cache {
-    DocTermsCache(FieldCacheImpl wrapper) {
+  static final class BinaryDocValuesCache extends Cache {
+    BinaryDocValuesCache(FieldCacheImpl wrapper) {
       super(wrapper);
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
+    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField /* ignored */)
         throws IOException {
 
-      Terms terms = reader.terms(entryKey.field);
+      // TODO: would be nice to first check if DocTermsIndex
+      // was already cached for this field and then return
+      // that instead, to avoid insanity
+
+      final int maxDoc = reader.maxDoc();
+      Terms terms = reader.terms(key.field);
 
-      final float acceptableOverheadRatio = ((Float) entryKey.custom).floatValue();
+      final float acceptableOverheadRatio = ((Float) key.custom).floatValue();
 
-      final int termCountHardLimit = reader.maxDoc();
+      final int termCountHardLimit = maxDoc;
 
       // Holds the actual term data, expanded.
       final PagedBytes bytes = new PagedBytes(15);
@@ -1257,7 +1265,7 @@ class FieldCacheImpl implements FieldCac
         startBPV = 1;
       }
 
-      final GrowableWriter docToOffset = new GrowableWriter(startBPV, reader.maxDoc(), acceptableOverheadRatio);
+      final GrowableWriter docToOffset = new GrowableWriter(startBPV, maxDoc, acceptableOverheadRatio);
       
       // pointer==0 means not set
       bytes.copyUsingLengthPrefix(new BytesRef());
@@ -1291,13 +1299,12 @@ class FieldCacheImpl implements FieldCac
       }
 
       // maybe an int-only impl?
-      return new DocTermsImpl(bytes.freeze(true), docToOffset.getMutable());
+      return new BinaryDocValuesImpl(bytes.freeze(true), docToOffset.getMutable());
     }
   }
 
-  @Override
   public DocTermOrds getDocTermOrds(AtomicReader reader, String field) throws IOException {
-    return (DocTermOrds) caches.get(DocTermOrds.class).get(reader, new Entry(field, null), false);
+    return (DocTermOrds) caches.get(DocTermOrds.class).get(reader, new CacheKey(field, null), false);
   }
 
   static final class DocTermOrdsCache extends Cache {
@@ -1306,20 +1313,19 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
+    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField /* ignored */)
         throws IOException {
-      return new DocTermOrds(reader, entryKey.field);
+      // No DocValues impl yet (DocValues are single valued...):
+      return new DocTermOrds(reader, key.field);
     }
   }
 
   private volatile PrintStream infoStream;
 
-  @Override
   public void setInfoStream(PrintStream stream) {
     infoStream = stream;
   }
 
-  @Override
   public PrintStream getInfoStream() {
     return infoStream;
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java Thu Feb  7 20:48:21 2013
@@ -18,15 +18,16 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.AtomicReader; // for javadocs
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.document.IntField; // for javadocs
+import org.apache.lucene.document.DoubleField; // for javadocs
 import org.apache.lucene.document.FloatField; // for javadocs
+import org.apache.lucene.document.IntField; // for javadocs
 import org.apache.lucene.document.LongField; // for javadocs
-import org.apache.lucene.document.DoubleField; // for javadocs
-import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.index.AtomicReader; // for javadocs
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
 
 /**
  * A range filter built on top of a cached single term field (in {@link FieldCache}).
@@ -89,43 +90,98 @@ public abstract class FieldCacheRangeFil
     return new FieldCacheRangeFilter<String>(field, null, lowerVal, upperVal, includeLower, includeUpper) {
       @Override
       public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
-        final FieldCache.DocTermsIndex fcsi = FieldCache.DEFAULT.getTermsIndex(context.reader(), field);
-        final BytesRef spare = new BytesRef();
-        final int lowerPoint = fcsi.binarySearchLookup(lowerVal == null ? null : new BytesRef(lowerVal), spare);
-        final int upperPoint = fcsi.binarySearchLookup(upperVal == null ? null : new BytesRef(upperVal), spare);
+        final SortedDocValues fcsi = FieldCache.DEFAULT.getTermsIndex(context.reader(), field);
+        final int lowerPoint = lowerVal == null ? -1 : fcsi.lookupTerm(new BytesRef(lowerVal));
+        final int upperPoint = upperVal == null ? -1 : fcsi.lookupTerm(new BytesRef(upperVal));
+
+        final int inclusiveLowerPoint, inclusiveUpperPoint;
+
+        // Hints:
+        // * binarySearchLookup returns -1, if value was null.
+        // * the value is <0 if no exact hit was found, the returned value
+        //   is (-(insertion point) - 1)
+        if (lowerPoint == -1 && lowerVal == null) {
+          inclusiveLowerPoint = 0;
+        } else if (includeLower && lowerPoint >= 0) {
+          inclusiveLowerPoint = lowerPoint;
+        } else if (lowerPoint >= 0) {
+          inclusiveLowerPoint = lowerPoint + 1;
+        } else {
+          inclusiveLowerPoint = Math.max(0, -lowerPoint - 1);
+        }
+        
+        if (upperPoint == -1 && upperVal == null) {
+          inclusiveUpperPoint = Integer.MAX_VALUE;  
+        } else if (includeUpper && upperPoint >= 0) {
+          inclusiveUpperPoint = upperPoint;
+        } else if (upperPoint >= 0) {
+          inclusiveUpperPoint = upperPoint - 1;
+        } else {
+          inclusiveUpperPoint = -upperPoint - 2;
+        }      
+
+        if (inclusiveUpperPoint < 0 || inclusiveLowerPoint > inclusiveUpperPoint) {
+          return DocIdSet.EMPTY_DOCIDSET;
+        }
+        
+        assert inclusiveLowerPoint >= 0 && inclusiveUpperPoint >= 0;
+        
+        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
+          @Override
+          protected final boolean matchDoc(int doc) {
+            final int docOrd = fcsi.getOrd(doc);
+            return docOrd >= inclusiveLowerPoint && docOrd <= inclusiveUpperPoint;
+          }
+        };
+      }
+    };
+  }
+  
+  /**
+   * Creates a BytesRef range filter using {@link FieldCache#getTermsIndex}. This works with all
+   * fields containing zero or one term in the field. The range can be half-open by setting one
+   * of the values to <code>null</code>.
+   */
+  // TODO: bogus that newStringRange doesnt share this code... generics hell
+  public static FieldCacheRangeFilter<BytesRef> newBytesRefRange(String field, BytesRef lowerVal, BytesRef upperVal, boolean includeLower, boolean includeUpper) {
+    return new FieldCacheRangeFilter<BytesRef>(field, null, lowerVal, upperVal, includeLower, includeUpper) {
+      @Override
+      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
+        final SortedDocValues fcsi = FieldCache.DEFAULT.getTermsIndex(context.reader(), field);
+        final int lowerPoint = lowerVal == null ? -1 : fcsi.lookupTerm(lowerVal);
+        final int upperPoint = upperVal == null ? -1 : fcsi.lookupTerm(upperVal);
 
         final int inclusiveLowerPoint, inclusiveUpperPoint;
 
         // Hints:
-        // * binarySearchLookup returns 0, if value was null.
+        // * binarySearchLookup returns -1, if value was null.
         // * the value is <0 if no exact hit was found, the returned value
         //   is (-(insertion point) - 1)
-        if (lowerPoint == 0) {
-          assert lowerVal == null;
-          inclusiveLowerPoint = 1;
-        } else if (includeLower && lowerPoint > 0) {
+        if (lowerPoint == -1 && lowerVal == null) {
+          inclusiveLowerPoint = 0;
+        } else if (includeLower && lowerPoint >= 0) {
           inclusiveLowerPoint = lowerPoint;
-        } else if (lowerPoint > 0) {
+        } else if (lowerPoint >= 0) {
           inclusiveLowerPoint = lowerPoint + 1;
         } else {
-          inclusiveLowerPoint = Math.max(1, -lowerPoint - 1);
+          inclusiveLowerPoint = Math.max(0, -lowerPoint - 1);
         }
         
-        if (upperPoint == 0) {
-          assert upperVal == null;
+        if (upperPoint == -1 && upperVal == null) {
           inclusiveUpperPoint = Integer.MAX_VALUE;  
-        } else if (includeUpper && upperPoint > 0) {
+        } else if (includeUpper && upperPoint >= 0) {
           inclusiveUpperPoint = upperPoint;
-        } else if (upperPoint > 0) {
+        } else if (upperPoint >= 0) {
           inclusiveUpperPoint = upperPoint - 1;
         } else {
           inclusiveUpperPoint = -upperPoint - 2;
         }      
 
-        if (inclusiveUpperPoint <= 0 || inclusiveLowerPoint > inclusiveUpperPoint)
+        if (inclusiveUpperPoint < 0 || inclusiveLowerPoint > inclusiveUpperPoint) {
           return DocIdSet.EMPTY_DOCIDSET;
+        }
         
-        assert inclusiveLowerPoint > 0 && inclusiveUpperPoint > 0;
+        assert inclusiveLowerPoint >= 0 && inclusiveUpperPoint >= 0;
         
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
@@ -177,11 +233,12 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final byte[] values = FieldCache.DEFAULT.getBytes(context.reader(), field, (FieldCache.ByteParser) parser, false);
+        final FieldCache.Bytes values = FieldCache.DEFAULT.getBytes(context.reader(), field, (FieldCache.ByteParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
-            return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
+            final byte value = values.get(doc);
+            return value >= inclusiveLowerPoint && value <= inclusiveUpperPoint;
           }
         };
       }
@@ -227,11 +284,12 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final short[] values = FieldCache.DEFAULT.getShorts(context.reader(), field, (FieldCache.ShortParser) parser, false);
+        final FieldCache.Shorts values = FieldCache.DEFAULT.getShorts(context.reader(), field, (FieldCache.ShortParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
-            return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
+            final short value = values.get(doc);
+            return value >= inclusiveLowerPoint && value <= inclusiveUpperPoint;
           }
         };
       }
@@ -277,11 +335,12 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final int[] values = FieldCache.DEFAULT.getInts(context.reader(), field, (FieldCache.IntParser) parser, false);
+        final FieldCache.Ints values = FieldCache.DEFAULT.getInts(context.reader(), field, (FieldCache.IntParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
-            return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
+            final int value = values.get(doc);
+            return value >= inclusiveLowerPoint && value <= inclusiveUpperPoint;
           }
         };
       }
@@ -327,11 +386,12 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final long[] values = FieldCache.DEFAULT.getLongs(context.reader(), field, (FieldCache.LongParser) parser, false);
+        final FieldCache.Longs values = FieldCache.DEFAULT.getLongs(context.reader(), field, (FieldCache.LongParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
-            return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
+            final long value = values.get(doc);
+            return value >= inclusiveLowerPoint && value <= inclusiveUpperPoint;
           }
         };
       }
@@ -381,11 +441,12 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final float[] values = FieldCache.DEFAULT.getFloats(context.reader(), field, (FieldCache.FloatParser) parser, false);
+        final FieldCache.Floats values = FieldCache.DEFAULT.getFloats(context.reader(), field, (FieldCache.FloatParser) parser, false);
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
-            return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
+            final float value = values.get(doc);
+            return value >= inclusiveLowerPoint && value <= inclusiveUpperPoint;
           }
         };
       }
@@ -435,12 +496,13 @@ public abstract class FieldCacheRangeFil
         if (inclusiveLowerPoint > inclusiveUpperPoint)
           return DocIdSet.EMPTY_DOCIDSET;
         
-        final double[] values = FieldCache.DEFAULT.getDoubles(context.reader(), field, (FieldCache.DoubleParser) parser, false);
+        final FieldCache.Doubles values = FieldCache.DEFAULT.getDoubles(context.reader(), field, (FieldCache.DoubleParser) parser, false);
         // ignore deleted docs if range doesn't contain 0
         return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
           @Override
           protected boolean matchDoc(int doc) {
-            return values[doc] >= inclusiveLowerPoint && values[doc] <= inclusiveUpperPoint;
+            final double value = values.get(doc);
+            return value >= inclusiveLowerPoint && value <= inclusiveUpperPoint;
           }
         };
       }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java Thu Feb  7 20:48:21 2013
@@ -22,9 +22,10 @@ import java.io.IOException;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocsEnum; // javadoc @link
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
 
 /**
  * A {@link Filter} that only accepts documents whose single
@@ -43,7 +44,7 @@ import org.apache.lucene.util.BytesRef;
  * <p/>
  * 
  * The first invocation of this filter on a given field will
- * be slower, since a {@link FieldCache.DocTermsIndex} must be
+ * be slower, since a {@link SortedDocValues} must be
  * created.  Subsequent invocations using the same field
  * will re-use this cache.  However, as with all
  * functionality based on {@link FieldCache}, persistent RAM
@@ -118,19 +119,24 @@ public class FieldCacheTermsFilter exten
 
   @Override
   public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
-    final FieldCache.DocTermsIndex fcsi = getFieldCache().getTermsIndex(context.reader(), field);
-    final FixedBitSet bits = new FixedBitSet(fcsi.numOrd());
-    final BytesRef spare = new BytesRef();
+    final SortedDocValues fcsi = getFieldCache().getTermsIndex(context.reader(), field);
+    final FixedBitSet bits = new FixedBitSet(fcsi.getValueCount());
     for (int i=0;i<terms.length;i++) {
-      int termNumber = fcsi.binarySearchLookup(terms[i], spare);
-      if (termNumber > 0) {
-        bits.set(termNumber);
+      int ord = fcsi.lookupTerm(terms[i]);
+      if (ord >= 0) {
+        bits.set(ord);
       }
     }
     return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
       @Override
       protected final boolean matchDoc(int doc) {
-        return bits.get(fcsi.getOrd(doc));
+        int ord = fcsi.getOrd(doc);
+        if (ord == -1) {
+          // missing
+          return false;
+        } else {
+          return bits.get(ord);
+        }
       }
     };
   }