You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/11/18 01:04:37 UTC

svn commit: r1410804 [1/2] - in /lucene/dev/branches/lucene4547/lucene: codecs/src/java/org/apache/lucene/codecs/simpletext/ core/src/java/org/apache/lucene/codecs/lucene41/ core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lucene/search/...

Author: mikemccand
Date: Sun Nov 18 00:04:31 2012
New Revision: 1410804

URL: http://svn.apache.org/viewvc?rev=1410804&view=rev
Log:
phase 2a

Modified:
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestSort.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
    lucene/dev/branches/lucene4547/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
    lucene/dev/branches/lucene4547/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ByteFieldSource.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/FloatFieldSource.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/IntFieldSource.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/LongFieldSource.java
    lucene/dev/branches/lucene4547/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ShortFieldSource.java
    lucene/dev/branches/lucene4547/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java Sun Nov 18 00:04:31 2012
@@ -94,8 +94,7 @@ public final class SimpleTextCodec exten
   private final SimpleDocValuesFormat nocommit = new SimpleTextSimpleDocValuesFormat();
   @Override
   public SimpleDocValuesFormat simpleDocValuesFormat() {
+    // nocommit
     return nocommit;
   }
-  
-  
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java Sun Nov 18 00:04:31 2012
@@ -124,7 +124,9 @@ public class Lucene41Codec extends Codec
   
   @Override
   public SimpleDocValuesFormat simpleDocValuesFormat() {
-    return simpleDocValuesFormat;
+    // nocommit fixme
+    //return simpleDocValuesFormat;
+    return null;
   }
 
   private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Sun Nov 18 00:04:31 2012
@@ -30,7 +30,6 @@ import org.apache.lucene.codecs.FieldInf
 import org.apache.lucene.codecs.PerDocConsumer;
 import org.apache.lucene.codecs.SimpleDVConsumer;
 import org.apache.lucene.codecs.SimpleDocValuesFormat;
-import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
 import org.apache.lucene.index.TypePromoter.TypeCompatibility;
 import org.apache.lucene.store.IOContext;
@@ -104,6 +103,10 @@ final class DocFieldProcessor extends Do
             }
 
             dvConsumer = fmt.fieldsConsumer(state);
+            // nocommit shouldn't need null check:
+            if (dvConsumer == null) {
+              continue;
+            }
           }
 
           if (field.bytesDVWriter != null) {
@@ -114,6 +117,7 @@ final class DocFieldProcessor extends Do
             // nocommit must null it out now else next seg
             // will flush even if no docs had DV...?
           }
+
           if (field.sortedBytesDVWriter != null) {
             field.sortedBytesDVWriter.flush(field.fieldInfo, state,
                                             dvConsumer.addSortedField(field.fieldInfo,
@@ -123,6 +127,7 @@ final class DocFieldProcessor extends Do
             // nocommit must null it out now else next seg
             // will flush even if no docs had DV...?
           }
+
           if (field.numberDVWriter != null) {
             field.numberDVWriter.flush(field.fieldInfo, state,
                                        dvConsumer.addNumericField(field.fieldInfo,

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Sun Nov 18 00:04:31 2012
@@ -117,8 +117,16 @@ final class SegmentCoreReaders {
       // kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
       norms = codec.normsFormat().docsProducer(segmentReadState);
       perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState);
-      // nocommit
-      simpleDVProducer = codec.simpleDocValuesFormat().fieldsProducer(segmentReadState);
+      // nocommit shouldn't need null check:
+      if (codec.simpleDocValuesFormat() != null) {
+        if (fieldInfos.hasDocValues()) {
+          simpleDVProducer = codec.simpleDocValuesFormat().fieldsProducer(segmentReadState);
+        } else {
+          simpleDVProducer = null;
+        }
+      } else {
+        simpleDVProducer = null;
+      }
   
       fieldsReaderOrig = si.info.getCodec().storedFieldsFormat().fieldsReader(cfsDir, si.info, fieldInfos, context);
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Sun Nov 18 00:04:31 2012
@@ -32,7 +32,6 @@ import org.apache.lucene.codecs.StoredFi
 import org.apache.lucene.codecs.TermVectorsWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
 
@@ -113,9 +112,12 @@ final class SegmentMerger {
     }
 
     if (mergeState.fieldInfos.hasDocValues()) {
-      SimpleDVConsumer consumer = codec.simpleDocValuesFormat().fieldsConsumer(segmentWriteState);
-      consumer.merge(mergeState);
-      consumer.close();
+      // nocommit shouldn't need null check:
+      if (codec.simpleDocValuesFormat() != null) {
+        SimpleDVConsumer consumer = codec.simpleDocValuesFormat().fieldsConsumer(segmentWriteState);
+        consumer.merge(mergeState);
+        consumer.close();
+      }
     }
 
     if (mergeState.fieldInfos.hasVectors()) {

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedBytesDVWriter.java Sun Nov 18 00:04:31 2012
@@ -18,12 +18,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter.DEFAULT;
-
-import org.apache.lucene.codecs.BinaryDocValuesConsumer;
 import org.apache.lucene.codecs.SortedDocValuesConsumer;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.ByteBlockPool;
@@ -31,7 +26,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 
 
@@ -104,7 +98,6 @@ class SortedBytesDVWriter {
   }
 
   public void flush(FieldInfo fieldInfo, SegmentWriteState state, SortedDocValuesConsumer consumer) throws IOException {
-    int valueCount = hash.size();
 
     final int maxDoc = state.segmentInfo.getDocCount();
     int emptyOrd = -1;
@@ -116,6 +109,8 @@ class SortedBytesDVWriter {
       }
     }
 
+    int valueCount = hash.size();
+
     int[] sortedValues = hash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
     final int sortedValueRamUsage = RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + RamUsageEstimator.NUM_BYTES_INT*valueCount;
     iwBytesUsed.addAndGet(sortedValueRamUsage);

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java Sun Nov 18 00:04:31 2012
@@ -29,4 +29,6 @@ public abstract class SortedDocValues {
 
   // nocommit throws IOE or not?
   public abstract int getValueCount();
+
+  // nocommit binary search lookup?
 }

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCache.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCache.java Sun Nov 18 00:04:31 2012
@@ -19,7 +19,6 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.io.PrintStream;
-import java.text.DecimalFormat;
 
 import org.apache.lucene.analysis.NumericTokenStream; // for javadocs
 import org.apache.lucene.document.IntField; // for javadocs
@@ -45,6 +44,30 @@ import org.apache.lucene.util.packed.Pac
  */
 public interface FieldCache {
 
+  public static abstract class Bytes {
+    public abstract byte get(int docID);
+  }
+
+  public static abstract class Shorts {
+    public abstract short get(int docID);
+  }
+
+  public static abstract class Ints {
+    public abstract int get(int docID);
+  }
+
+  public static abstract class Longs {
+    public abstract long get(int docID);
+  }
+
+  public static abstract class Floats {
+    public abstract float get(int docID);
+  }
+
+  public static abstract class Doubles {
+    public abstract double get(int docID);
+  }
+
   /**
    * Placeholder indicating creation of this cache is currently in-progress.
    */
@@ -294,8 +317,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public byte[] getBytes (AtomicReader reader, String field, boolean setDocsWithField)
-  throws IOException;
+  public Bytes getBytes(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is found,
    * reads the terms in <code>field</code> as bytes and returns an array of
@@ -309,8 +331,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public byte[] getBytes (AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField)
-  throws IOException;
+  public Bytes getBytes(AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is
    * found, reads the terms in <code>field</code> as shorts and returns an array
@@ -323,8 +344,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public short[] getShorts (AtomicReader reader, String field, boolean setDocsWithField)
-  throws IOException;
+  public Shorts getShorts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is found,
    * reads the terms in <code>field</code> as shorts and returns an array of
@@ -338,8 +358,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public short[] getShorts (AtomicReader reader, String field, ShortParser parser, boolean setDocsWithField)
-  throws IOException;
+  public Shorts getShorts (AtomicReader reader, String field, ShortParser parser, boolean setDocsWithField) throws IOException;
   
   /** Checks the internal cache for an appropriate entry, and if none is
    * found, reads the terms in <code>field</code> as integers and returns an array
@@ -352,8 +371,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public int[] getInts (AtomicReader reader, String field, boolean setDocsWithField)
-  throws IOException;
+  public Ints getInts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is found,
    * reads the terms in <code>field</code> as integers and returns an array of
@@ -367,8 +385,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public int[] getInts (AtomicReader reader, String field, IntParser parser, boolean setDocsWithField)
-  throws IOException;
+  public Ints getInts (AtomicReader reader, String field, IntParser parser, boolean setDocsWithField) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if
    * none is found, reads the terms in <code>field</code> as floats and returns an array
@@ -381,8 +398,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public float[] getFloats (AtomicReader reader, String field, boolean setDocsWithField)
-  throws IOException;
+  public Floats getFloats (AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if
    * none is found, reads the terms in <code>field</code> as floats and returns an array
@@ -396,8 +412,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public float[] getFloats (AtomicReader reader, String field,
-                            FloatParser parser, boolean setDocsWithField) throws IOException;
+  public Floats getFloats (AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField) throws IOException;
 
   /**
    * Checks the internal cache for an appropriate entry, and if none is
@@ -412,7 +427,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws java.io.IOException If any error occurs.
    */
-  public long[] getLongs(AtomicReader reader, String field, boolean setDocsWithField)
+  public Longs getLongs(AtomicReader reader, String field, boolean setDocsWithField)
           throws IOException;
 
   /**
@@ -429,7 +444,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException If any error occurs.
    */
-  public long[] getLongs(AtomicReader reader, String field, LongParser parser, boolean setDocsWithField)
+  public Longs getLongs(AtomicReader reader, String field, LongParser parser, boolean setDocsWithField)
           throws IOException;
 
   /**
@@ -445,7 +460,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException If any error occurs.
    */
-  public double[] getDoubles(AtomicReader reader, String field, boolean setDocsWithField)
+  public Doubles getDoubles(AtomicReader reader, String field, boolean setDocsWithField)
           throws IOException;
 
   /**
@@ -462,7 +477,7 @@ public interface FieldCache {
    * @return The values in the given field for each document.
    * @throws IOException If any error occurs.
    */
-  public double[] getDoubles(AtomicReader reader, String field, DoubleParser parser, boolean setDocsWithField)
+  public Doubles getDoubles(AtomicReader reader, String field, DoubleParser parser, boolean setDocsWithField)
           throws IOException;
 
   /** Returned by {@link #getTerms} */

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Sun Nov 18 00:04:31 2012
@@ -45,6 +45,8 @@ import org.apache.lucene.util.PagedBytes
 import org.apache.lucene.util.packed.GrowableWriter;
 import org.apache.lucene.util.packed.PackedInts;
 
+// nocommit rename to UninvertFieldCache or something ...
+
 /**
  * Expert: The default cache implementation, storing all values in memory.
  * A WeakHashMap is used for storage.
@@ -137,14 +139,6 @@ class FieldCacheImpl implements FieldCac
     public Object getValue() { return value; }
   }
 
-  /**
-   * Hack: When thrown from a Parser (NUMERIC_UTILS_* ones), this stops
-   * processing terms and returns the current FieldCache
-   * array.
-   */
-  static final class StopFillCacheException extends RuntimeException {
-  }
-  
   // per-segment fieldcaches don't purge until the shared core closes.
   final SegmentReader.CoreClosedListener purgeCore = new SegmentReader.CoreClosedListener() {
     @Override
@@ -181,6 +175,7 @@ class FieldCacheImpl implements FieldCac
   /** Expert: Internal cache. */
   abstract static class Cache {
 
+    // nocommit why wrapper vs non-static class...?
     Cache(FieldCacheImpl wrapper) {
       this.wrapper = wrapper;
     }
@@ -319,86 +314,153 @@ class FieldCacheImpl implements FieldCac
   }
 
   // inherit javadocs
-  public byte[] getBytes (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
+  public Bytes getBytes (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
     return getBytes(reader, field, null, setDocsWithField);
   }
 
   // inherit javadocs
-  public byte[] getBytes(AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField)
+  public Bytes getBytes(AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField)
       throws IOException {
-    return (byte[]) caches.get(Byte.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+    return (Bytes) caches.get(Byte.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
   }
 
-  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);
-      }
+  private static abstract class Uninvert {
+
+    public Bits docsWithField;
+
+    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 = terms.iterator(null);
+
         DocsEnum docs = null;
-        try {
-          while(true) {
-            final BytesRef term = termsEnum.next();
-            if (term == null) {
+        FixedBitSet docsWithField = null;
+        while(true) {
+          final BytesRef term = termsEnum.next();
+          if (term == null) {
+            break;
+          }
+          try {
+            visitTerm(term);
+          } catch (StopFillCacheException stop) {
+            break;
+          }
+          docs = termsEnum.docs(null, docs, 0);
+          while (true) {
+            final int docID = docs.nextDoc();
+            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
             }
-            final byte termval = parser.parseByte(term);
-            docs = termsEnum.docs(null, docs, 0);
-            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);
+            visitDoc(docID);
+            if (setDocsWithField) {
+              if (docsWithField == null) {
+                // Lazy init
+                this.docsWithField = docsWithField = new FixedBitSet(maxDoc);
               }
+              docsWithField.set(docID);
             }
           }
-        } catch (FieldCache.StopFillCacheException stop) {
         }
+      } else {
+        // nocommit is this right ...
+        docsWithField = new Bits.MatchNoBits(maxDoc);
       }
+    }
+
+    protected abstract void visitTerm(BytesRef term);
+    protected abstract void visitDoc(int docID);
+  }
+
+  private 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, Entry entryKey, boolean setDocsWithField)
+        throws IOException {
+
+      ByteParser parser = (ByteParser) entryKey.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, entryKey.field, DEFAULT_BYTE_PARSER, setDocsWithField);
+      }
+
+      final ByteParser finalParser = parser;
+
+      final byte[] values = new byte[reader.maxDoc()];
+      Uninvert u = new Uninvert() {
+        private byte currentValue;
+
+        @Override
+        public void visitTerm(BytesRef term) {
+          currentValue = finalParser.parseByte(term);
+        }
+
+        @Override
+        public void visitDoc(int docID) {
+          values[docID] = currentValue;
+        }
+      };
+
+      u.uninvert(reader, entryKey.field, setDocsWithField);
+
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, entryKey.field, u.docsWithField);
       }
-      return retArray;
+
+      return new BytesFromArray(values);
     }
   }
   
   // inherit javadocs
-  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
-  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);
+    return (Shorts) caches.get(Short.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+  }
+
+  private 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 {
@@ -409,57 +471,39 @@ class FieldCacheImpl implements FieldCac
     @Override
     protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
+
       ShortParser parser = (ShortParser) entryKey.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, entryKey.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;
-          }
+
+      final ShortParser finalParser = parser;
+
+      final short[] values = new short[reader.maxDoc()];
+      Uninvert u = new Uninvert() {
+        private short currentValue;
+
+        @Override
+        public void visitTerm(BytesRef term) {
+          currentValue = finalParser.parseShort(term);
         }
-        final TermsEnum termsEnum = terms.iterator(null);
-        DocsEnum docs = null;
-        try {
-          while(true) {
-            final BytesRef term = termsEnum.next();
-            if (term == null) {
-              break;
-            }
-            final short termval = parser.parseShort(term);
-            docs = termsEnum.docs(null, docs, 0);
-            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);
-              }
-            }
-          }
-        } catch (FieldCache.StopFillCacheException stop) {
+
+        @Override
+        public void visitDoc(int docID) {
+          values[docID] = currentValue;
         }
-      }
+      };
+
+      u.uninvert(reader, entryKey.field, setDocsWithField);
+
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, entryKey.field, u.docsWithField);
       }
-      return retArray;
+
+      return new ShortsFromArray(values);
     }
   }
 
@@ -485,14 +529,27 @@ class FieldCacheImpl implements FieldCac
   }
   
   // inherit javadocs
-  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
-  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);
+    return (Ints) caches.get(Integer.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+  }
+
+  private 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];
+    }
   }
 
   static final class IntCache extends Cache {
@@ -501,74 +558,48 @@ class FieldCacheImpl implements FieldCac
     }
 
     @Override
-    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
+    protected Object createValue(final AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
       IntParser parser = (IntParser) entryKey.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, entryKey.field, DEFAULT_INT_PARSER, setDocsWithField);
         } catch (NumberFormatException ne) {
-          return wrapper.getInts(reader, field, NUMERIC_UTILS_INT_PARSER, setDocsWithField);
+          return wrapper.getInts(reader, entryKey.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 IntParser finalParser = parser;
+      // nocommit how to avoid double alloc in numeric field
+      // case ...
+      final int[] values = new int[reader.maxDoc()];
+
+      Uninvert u = new Uninvert() {
+        private int currentValue;
+
+        @Override
+        public void visitTerm(BytesRef term) {
+          currentValue = finalParser.parseInt(term);
         }
-        final TermsEnum termsEnum = terms.iterator(null);
-        DocsEnum docs = null;
-        try {
-          while(true) {
-            final BytesRef term = termsEnum.next();
-            if (term == null) {
-              break;
-            }
-            final int termval = parser.parseInt(term);
-            if (retArray == null) {
-              // late init so numeric fields don't double allocate
-              retArray = new int[maxDoc];
-            }
 
-            docs = termsEnum.docs(null, docs, 0);
-            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);
-              }
-            }
-          }
-        } catch (FieldCache.StopFillCacheException stop) {
+        @Override
+        public void visitDoc(int docID) {
+          values[docID] = currentValue;
         }
-      }
+      };
+
+      u.uninvert(reader, entryKey.field, setDocsWithField);
 
-      if (retArray == null) {
-        // no values
-        retArray = new int[maxDoc];
-      }
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, entryKey.field, u.docsWithField);
       }
-      return retArray;
+
+      return new IntsFromArray(values);
     }
   }
   
@@ -583,7 +614,7 @@ class FieldCacheImpl implements FieldCac
     }
     
     @Override
-      protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
+    protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField /* ignored */)
     throws IOException {
       final String field = entryKey.field;      
       FixedBitSet res = null;
@@ -633,16 +664,28 @@ class FieldCacheImpl implements FieldCac
   }
 
   // inherit javadocs
-  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
-  public float[] getFloats(AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField)
+  public Floats getFloats(AtomicReader reader, String field, FloatParser parser, boolean setDocsWithField)
     throws IOException {
+    return (Floats) caches.get(Float.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+  }
+
+  private 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 {
@@ -653,84 +696,71 @@ class FieldCacheImpl implements FieldCac
     @Override
     protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
       FloatParser parser = (FloatParser) entryKey.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, entryKey.field, DEFAULT_FLOAT_PARSER, setDocsWithField);
         } catch (NumberFormatException ne) {
-          return wrapper.getFloats(reader, field, NUMERIC_UTILS_FLOAT_PARSER, setDocsWithField);
+          return wrapper.getFloats(reader, entryKey.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 FloatParser finalParser = parser;
+      // nocommit how to avoid double alloc in numeric field
+      // case ...
+      final float[] values = new float[reader.maxDoc()];
+
+      Uninvert u = new Uninvert() {
+        private float currentValue;
+
+        @Override
+        public void visitTerm(BytesRef term) {
+          currentValue = finalParser.parseFloat(term);
         }
-        final TermsEnum termsEnum = terms.iterator(null);
-        DocsEnum docs = null;
-        try {
-          while(true) {
-            final BytesRef term = termsEnum.next();
-            if (term == null) {
-              break;
-            }
-            final float termval = parser.parseFloat(term);
-            if (retArray == null) {
-              // late init so numeric fields don't double allocate
-              retArray = new float[maxDoc];
-            }
-            
-            docs = termsEnum.docs(null, docs, 0);
-            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);
-              }
-            }
-          }
-        } catch (FieldCache.StopFillCacheException stop) {
+
+        @Override
+        public void visitDoc(int docID) {
+          values[docID] = currentValue;
         }
-      }
+      };
+
+      u.uninvert(reader, entryKey.field, setDocsWithField);
 
-      if (retArray == null) {
-        // no values
-        retArray = new float[maxDoc];
-      }
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, entryKey.field, u.docsWithField);
       }
-      return retArray;
+
+      return new FloatsFromArray(values);
     }
   }
 
-
-  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
-  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);
+    return (Longs) caches.get(Long.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+  }
+
+  private 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 {
@@ -741,85 +771,72 @@ class FieldCacheImpl implements FieldCac
     @Override
     protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
-      FieldCache.LongParser parser = (FieldCache.LongParser) entryKey.custom;
+      LongParser parser = (LongParser) entryKey.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, entryKey.field, DEFAULT_LONG_PARSER, setDocsWithField);
         } catch (NumberFormatException ne) {
-          return wrapper.getLongs(reader, field, NUMERIC_UTILS_LONG_PARSER, setDocsWithField);
+          return wrapper.getLongs(reader, entryKey.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 LongParser finalParser = parser;
+      // nocommit how to avoid double alloc in numeric field
+      // case ...
+      final long[] values = new long[reader.maxDoc()];
+
+      Uninvert u = new Uninvert() {
+        private long currentValue;
+
+        @Override
+        public void visitTerm(BytesRef term) {
+          currentValue = finalParser.parseLong(term);
         }
-        final TermsEnum termsEnum = terms.iterator(null);
-        DocsEnum docs = null;
-        try {
-          while(true) {
-            final BytesRef term = termsEnum.next();
-            if (term == null) {
-              break;
-            }
-            final long termval = parser.parseLong(term);
-            if (retArray == null) {
-              // late init so numeric fields don't double allocate
-              retArray = new long[maxDoc];
-            }
 
-            docs = termsEnum.docs(null, docs, 0);
-            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);
-              }
-            }
-          }
-        } catch (FieldCache.StopFillCacheException stop) {
+        @Override
+        public void visitDoc(int docID) {
+          values[docID] = currentValue;
         }
-      }
+      };
+
+      u.uninvert(reader, entryKey.field, setDocsWithField);
 
-      if (retArray == null) {
-        // no values
-        retArray = new long[maxDoc];
-      }
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, entryKey.field, u.docsWithField);
       }
-      return retArray;
+
+      return new LongsFromArray(values);
     }
   }
 
   // inherit javadocs
-  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
-  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);
+    return (Doubles) caches.get(Double.TYPE).get(reader, new Entry(field, parser), setDocsWithField);
+  }
+
+  private 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 {
@@ -830,70 +847,46 @@ class FieldCacheImpl implements FieldCac
     @Override
     protected Object createValue(AtomicReader reader, Entry entryKey, boolean setDocsWithField)
         throws IOException {
-      String field = entryKey.field;
-      FieldCache.DoubleParser parser = (FieldCache.DoubleParser) entryKey.custom;
+      DoubleParser parser = (DoubleParser) entryKey.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, entryKey.field, DEFAULT_DOUBLE_PARSER, setDocsWithField);
         } catch (NumberFormatException ne) {
-          return wrapper.getDoubles(reader, field, NUMERIC_UTILS_DOUBLE_PARSER, setDocsWithField);
+          return wrapper.getDoubles(reader, entryKey.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 DoubleParser finalParser = parser;
+      // nocommit how to avoid double alloc in numeric field
+      // case ...
+      final double[] values = new double[reader.maxDoc()];
+
+      Uninvert u = new Uninvert() {
+        private double currentValue;
+
+        @Override
+        public void visitTerm(BytesRef term) {
+          currentValue = finalParser.parseDouble(term);
         }
-        final TermsEnum termsEnum = terms.iterator(null);
-        DocsEnum docs = null;
-        try {
-          while(true) {
-            final BytesRef term = termsEnum.next();
-            if (term == null) {
-              break;
-            }
-            final double termval = parser.parseDouble(term);
-            if (retArray == null) {
-              // late init so numeric fields don't double allocate
-              retArray = new double[maxDoc];
-            }
 
-            docs = termsEnum.docs(null, docs, 0);
-            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);
-              }
-            }
-          }
-        } catch (FieldCache.StopFillCacheException stop) {
+        @Override
+        public void visitDoc(int docID) {
+          values[docID] = currentValue;
         }
-      }
-      if (retArray == null) { // no values
-        retArray = new double[maxDoc];
-      }
+      };
+
+      u.uninvert(reader, entryKey.field, setDocsWithField);
+
       if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, field, docsWithField);
+        wrapper.setDocsWithField(reader, entryKey.field, u.docsWithField);
       }
-      return retArray;
+
+      return new DoublesFromArray(values);
     }
   }
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java Sun Nov 18 00:04:31 2012
@@ -177,11 +177,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 +228,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 +279,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 +330,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 +385,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 +440,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/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Sun Nov 18 00:04:31 2012
@@ -227,7 +227,7 @@ public abstract class FieldComparator<T>
   public static final class ByteComparator extends NumericComparator<Byte> {
     private final byte[] values;
     private final ByteParser parser;
-    private byte[] currentReaderValues;
+    private FieldCache.Bytes currentReaderValues;
     private byte bottom;
 
     ByteComparator(int numHits, String field, FieldCache.Parser parser, Byte missingValue) {
@@ -243,7 +243,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compareBottom(int doc) {
-      byte v2 = currentReaderValues[doc];
+      byte v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -255,7 +255,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public void copy(int slot, int doc) {
-      byte v2 = currentReaderValues[doc];
+      byte v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -284,7 +284,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compareDocToValue(int doc, Byte value) {
-      byte docValue = currentReaderValues[doc];
+      byte docValue = currentReaderValues.get(doc);
       // Test for docValue == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
@@ -299,7 +299,7 @@ public abstract class FieldComparator<T>
   public static final class DoubleComparator extends NumericComparator<Double> {
     private final double[] values;
     private final DoubleParser parser;
-    private double[] currentReaderValues;
+    private FieldCache.Doubles currentReaderValues;
     private double bottom;
 
     DoubleComparator(int numHits, String field, FieldCache.Parser parser, Double missingValue) {
@@ -323,7 +323,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compareBottom(int doc) {
-      double v2 = currentReaderValues[doc];
+      double v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -341,7 +341,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public void copy(int slot, int doc) {
-      double v2 = currentReaderValues[doc];
+      double v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -372,7 +372,7 @@ public abstract class FieldComparator<T>
     @Override
     public int compareDocToValue(int doc, Double valueObj) {
       final double value = valueObj.doubleValue();
-      double docValue = currentReaderValues[doc];
+      double docValue = currentReaderValues.get(doc);
       // Test for docValue == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
@@ -470,7 +470,7 @@ public abstract class FieldComparator<T>
   public static final class FloatComparator extends NumericComparator<Float> {
     private final float[] values;
     private final FloatParser parser;
-    private float[] currentReaderValues;
+    private FieldCache.Floats currentReaderValues;
     private float bottom;
 
     FloatComparator(int numHits, String field, FieldCache.Parser parser, Float missingValue) {
@@ -497,7 +497,7 @@ public abstract class FieldComparator<T>
     @Override
     public int compareBottom(int doc) {
       // TODO: are there sneaky non-branch ways to compute sign of float?
-      float v2 = currentReaderValues[doc];
+      float v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -515,7 +515,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public void copy(int slot, int doc) {
-      float v2 = currentReaderValues[doc];
+      float v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -546,7 +546,7 @@ public abstract class FieldComparator<T>
     @Override
     public int compareDocToValue(int doc, Float valueObj) {
       final float value = valueObj.floatValue();
-      float docValue = currentReaderValues[doc];
+      float docValue = currentReaderValues.get(doc);
       // Test for docValue == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
@@ -567,7 +567,7 @@ public abstract class FieldComparator<T>
   public static final class ShortComparator extends NumericComparator<Short> {
     private final short[] values;
     private final ShortParser parser;
-    private short[] currentReaderValues;
+    private FieldCache.Shorts currentReaderValues;
     private short bottom;
 
     ShortComparator(int numHits, String field, FieldCache.Parser parser, Short missingValue) {
@@ -583,7 +583,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public int compareBottom(int doc) {
-      short v2 = currentReaderValues[doc];
+      short v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -595,7 +595,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public void copy(int slot, int doc) {
-      short v2 = currentReaderValues[doc];
+      short v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -626,7 +626,7 @@ public abstract class FieldComparator<T>
     @Override
     public int compareDocToValue(int doc, Short valueObj) {
       final short value = valueObj.shortValue();
-      short docValue = currentReaderValues[doc];
+      short docValue = currentReaderValues.get(doc);
       // Test for docValue == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
@@ -641,7 +641,7 @@ public abstract class FieldComparator<T>
   public static final class IntComparator extends NumericComparator<Integer> {
     private final int[] values;
     private final IntParser parser;
-    private int[] currentReaderValues;
+    private FieldCache.Ints currentReaderValues;
     private int bottom;                           // Value of bottom of queue
 
     IntComparator(int numHits, String field, FieldCache.Parser parser, Integer missingValue) {
@@ -673,7 +673,7 @@ public abstract class FieldComparator<T>
       // -1/+1/0 sign
       // Cannot return bottom - values[slot2] because that
       // may overflow
-      int v2 = currentReaderValues[doc];
+      int v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -691,7 +691,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public void copy(int slot, int doc) {
-      int v2 = currentReaderValues[doc];
+      int v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -722,7 +722,7 @@ public abstract class FieldComparator<T>
     @Override
     public int compareDocToValue(int doc, Integer valueObj) {
       final int value = valueObj.intValue();
-      int docValue = currentReaderValues[doc];
+      int docValue = currentReaderValues.get(doc);
       // Test for docValue == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
@@ -824,7 +824,7 @@ public abstract class FieldComparator<T>
   public static final class LongComparator extends NumericComparator<Long> {
     private final long[] values;
     private final LongParser parser;
-    private long[] currentReaderValues;
+    private FieldCache.Longs currentReaderValues;
     private long bottom;
 
     LongComparator(int numHits, String field, FieldCache.Parser parser, Long missingValue) {
@@ -852,7 +852,7 @@ public abstract class FieldComparator<T>
     public int compareBottom(int doc) {
       // TODO: there are sneaky non-branch ways to compute
       // -1/+1/0 sign
-      long v2 = currentReaderValues[doc];
+      long v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -870,7 +870,7 @@ public abstract class FieldComparator<T>
 
     @Override
     public void copy(int slot, int doc) {
-      long v2 = currentReaderValues[doc];
+      long v2 = currentReaderValues.get(doc);
       // Test for v2 == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
@@ -901,7 +901,7 @@ public abstract class FieldComparator<T>
     @Override
     public int compareDocToValue(int doc, Long valueObj) {
       final long value = valueObj.longValue();
-      long docValue = currentReaderValues[doc];
+      long docValue = currentReaderValues.get(doc);
       // Test for docValue == 0 to save Bits.get method call for
       // the common case (doc has value and value is non-zero):
       if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java Sun Nov 18 00:04:31 2012
@@ -29,20 +29,18 @@ import org.apache.lucene.document.Straig
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.*;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase;
-import org.junit.Ignore;
 
 /**
  * A very simple demo used in the API documentation (src/java/overview.html).
@@ -50,6 +48,8 @@ import org.junit.Ignore;
  * Please try to keep src/java/overview.html up-to-date when making changes
  * to this class.
  */
+// nocommit don't suppress any:
+@SuppressCodecs({"Direct", "Memory", "Lucene41", "MockRandom"})
 public class TestDemoDocValue extends LuceneTestCase {
 
   public void testDemoNumber() throws IOException {

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Sun Nov 18 00:04:31 2012
@@ -58,7 +58,6 @@ import org.apache.lucene.search.ScoreDoc
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -810,13 +809,16 @@ public class TestBackwardsCompatibility 
       assertEquals("wrong number of hits", 34, hits.length);
       
       // check decoding into field cache
-      int[] fci = FieldCache.DEFAULT.getInts(SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()), "trieInt", false);
-      for (int val : fci) {
+      FieldCache.Ints fci = FieldCache.DEFAULT.getInts(SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()), "trieInt", false);
+      int maxDoc = searcher.getIndexReader().maxDoc();
+      for(int doc=0;doc<maxDoc;doc++) {
+        int val = fci.get(doc);
         assertTrue("value in id bounds", val >= 0 && val < 35);
       }
       
-      long[] fcl = FieldCache.DEFAULT.getLongs(SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()), "trieLong", false);
-      for (long val : fcl) {
+      FieldCache.Longs fcl = FieldCache.DEFAULT.getLongs(SlowCompositeReaderWrapper.wrap(searcher.getIndexReader()), "trieLong", false);
+      for(int doc=0;doc<maxDoc;doc++) {
+        long val = fcl.get(doc);
         assertTrue("value in id bounds", val >= 0L && val < 35L);
       }
       

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java Sun Nov 18 00:04:31 2012
@@ -39,7 +39,6 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.NoSuchDirectoryException;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -774,9 +773,8 @@ public void testFilesOpenClose() throws 
     // Open reader1
     DirectoryReader r = DirectoryReader.open(dir);
     AtomicReader r1 = getOnlySegmentReader(r);
-    final int[] ints = FieldCache.DEFAULT.getInts(r1, "number", false);
-    assertEquals(1, ints.length);
-    assertEquals(17, ints[0]);
+    final FieldCache.Ints ints = FieldCache.DEFAULT.getInts(r1, "number", false);
+    assertEquals(17, ints.get(0));
   
     // Add new segment
     writer.addDocument(doc);
@@ -787,7 +785,7 @@ public void testFilesOpenClose() throws 
     assertNotNull(r2);
     r.close();
     AtomicReader sub0 = r2.leaves().get(0).reader();
-    final int[] ints2 = FieldCache.DEFAULT.getInts(sub0, "number", false);
+    final FieldCache.Ints ints2 = FieldCache.DEFAULT.getInts(sub0, "number", false);
     r2.close();
     assertTrue(ints == ints2);
   

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java Sun Nov 18 00:04:31 2012
@@ -32,7 +32,6 @@ import org.apache.lucene.document.IntFie
 import org.apache.lucene.index.DocTermOrds.TermOrdsIterator;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.StringHelper;
@@ -303,7 +302,7 @@ public class TestDocTermOrds extends Luc
                                             _TestUtil.nextInt(random(), 2, 10));
                                             
 
-    final int[] docIDToID = FieldCache.DEFAULT.getInts(r, "id", false);
+    final FieldCache.Ints docIDToID = FieldCache.DEFAULT.getInts(r, "id", false);
     /*
       for(int docID=0;docID<subR.maxDoc();docID++) {
       System.out.println("  docID=" + docID + " id=" + docIDToID[docID]);
@@ -357,10 +356,10 @@ public class TestDocTermOrds extends Luc
     final int[] buffer = new int[5];
     for(int docID=0;docID<r.maxDoc();docID++) {
       if (VERBOSE) {
-        System.out.println("TEST: docID=" + docID + " of " + r.maxDoc() + " (id=" + docIDToID[docID] + ")");
+        System.out.println("TEST: docID=" + docID + " of " + r.maxDoc() + " (id=" + docIDToID.get(docID) + ")");
       }
       iter = dto.lookup(docID, iter);
-      final int[] answers = idToOrds[docIDToID[docID]];
+      final int[] answers = idToOrds[docIDToID.get(docID)];
       int upto = 0;
       while(true) {
         final int chunk = iter.read(buffer);

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java Sun Nov 18 00:04:31 2012
@@ -291,12 +291,12 @@ public class TestFieldsReader extends Lu
 
     for(AtomicReaderContext ctx : r.leaves()) {
       final AtomicReader sub = ctx.reader();
-      final int[] ids = FieldCache.DEFAULT.getInts(sub, "id", false);
+      final FieldCache.Ints ids = FieldCache.DEFAULT.getInts(sub, "id", false);
       for(int docID=0;docID<sub.numDocs();docID++) {
         final StoredDocument doc = sub.document(docID);
         final Field f = (Field) doc.getField("nf");
         assertTrue("got f=" + f, f instanceof StoredField);
-        assertEquals(answers[ids[docID]], f.numericValue());
+        assertEquals(answers[ids.get(docID)], f.numericValue());
       }
     }
     r.close();

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java Sun Nov 18 00:04:31 2012
@@ -297,7 +297,7 @@ public class TestPostingsOffsets extends
       DocsEnum docs = null;
       DocsAndPositionsEnum docsAndPositions = null;
       DocsAndPositionsEnum docsAndPositionsAndOffsets = null;
-      final int docIDToID[] = FieldCache.DEFAULT.getInts(sub, "id", false);
+      final FieldCache.Ints docIDToID = FieldCache.DEFAULT.getInts(sub, "id", false);
       for(String term : terms) {
         //System.out.println("  term=" + term);
         if (termsEnum.seekExact(new BytesRef(term), random().nextBoolean())) {
@@ -306,8 +306,8 @@ public class TestPostingsOffsets extends
           int doc;
           //System.out.println("    doc/freq");
           while((doc = docs.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-            final List<Token> expected = actualTokens.get(term).get(docIDToID[doc]);
-            //System.out.println("      doc=" + docIDToID[doc] + " docID=" + doc + " " + expected.size() + " freq");
+            final List<Token> expected = actualTokens.get(term).get(docIDToID.get(doc));
+            //System.out.println("      doc=" + docIDToID.get(doc) + " docID=" + doc + " " + expected.size() + " freq");
             assertNotNull(expected);
             assertEquals(expected.size(), docs.freq());
           }
@@ -317,8 +317,8 @@ public class TestPostingsOffsets extends
           assertNotNull(docsAndPositions);
           //System.out.println("    doc/freq/pos");
           while((doc = docsAndPositions.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-            final List<Token> expected = actualTokens.get(term).get(docIDToID[doc]);
-            //System.out.println("      doc=" + docIDToID[doc] + " " + expected.size() + " freq");
+            final List<Token> expected = actualTokens.get(term).get(docIDToID.get(doc));
+            //System.out.println("      doc=" + docIDToID.get(doc) + " " + expected.size() + " freq");
             assertNotNull(expected);
             assertEquals(expected.size(), docsAndPositions.freq());
             for(Token token : expected) {
@@ -332,8 +332,8 @@ public class TestPostingsOffsets extends
           assertNotNull(docsAndPositionsAndOffsets);
           //System.out.println("    doc/freq/pos/offs");
           while((doc = docsAndPositionsAndOffsets.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-            final List<Token> expected = actualTokens.get(term).get(docIDToID[doc]);
-            //System.out.println("      doc=" + docIDToID[doc] + " " + expected.size() + " freq");
+            final List<Token> expected = actualTokens.get(term).get(docIDToID.get(doc));
+            //System.out.println("      doc=" + docIDToID.get(doc) + " " + expected.size() + " freq");
             assertNotNull(expected);
             assertEquals(expected.size(), docsAndPositionsAndOffsets.freq());
             for(Token token : expected) {

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java?rev=1410804&r1=1410803&r2=1410804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java Sun Nov 18 00:04:31 2012
@@ -225,7 +225,7 @@ public class TestTermsEnum extends Lucen
     w.close();
 
     // NOTE: intentional insanity!!
-    final int[] docIDToID = FieldCache.DEFAULT.getInts(SlowCompositeReaderWrapper.wrap(r), "id", false);
+    final FieldCache.Ints docIDToID = FieldCache.DEFAULT.getInts(SlowCompositeReaderWrapper.wrap(r), "id", false);
 
     for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) {
 
@@ -335,7 +335,7 @@ public class TestTermsEnum extends Lucen
           docsEnum = _TestUtil.docs(random(), te, null, docsEnum, 0);
           final int docID = docsEnum.nextDoc();
           assertTrue(docID != DocIdSetIterator.NO_MORE_DOCS);
-          assertEquals(docIDToID[docID], termToID.get(expected).intValue());
+          assertEquals(docIDToID.get(docID), termToID.get(expected).intValue());
           do {
             loc++;
           } while (loc < termsArray.length && !acceptTermsSet.contains(termsArray[loc]));