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/08 04:26:21 UTC

svn commit: r1443834 [4/16] - in /lucene/dev/branches/branch_4x: ./ dev-tools/ lucene/ lucene/analysis/ lucene/analysis/icu/src/java/org/apache/lucene/collation/ lucene/analysis/icu/src/test/org/apache/lucene/collation/ lucene/backwards/ lucene/benchma...

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Fri Feb  8 03:26:14 2013
@@ -19,16 +19,19 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedHashSet;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.PerDocProducer;
 import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.SegmentReader.CoreClosedListener;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
@@ -51,8 +54,8 @@ final class SegmentCoreReaders {
   final FieldInfos fieldInfos;
   
   final FieldsProducer fields;
-  final PerDocProducer perDocProducer;
-  final PerDocProducer norms;
+  final DocValuesProducer dvProducer;
+  final DocValuesProducer normsProducer;
 
   final int termsIndexDivisor;
   
@@ -62,6 +65,10 @@ final class SegmentCoreReaders {
   final TermVectorsReader termVectorsReaderOrig;
   final CompoundFileDirectory cfsReader;
 
+  // TODO: make a single thread local w/ a
+  // Thingy class holding fieldsReader, termVectorsReader,
+  // normsProducer, dvProducer
+
   final CloseableThreadLocal<StoredFieldsReader> fieldsReaderLocal = new CloseableThreadLocal<StoredFieldsReader>() {
     @Override
     protected StoredFieldsReader initialValue() {
@@ -72,11 +79,24 @@ final class SegmentCoreReaders {
   final CloseableThreadLocal<TermVectorsReader> termVectorsLocal = new CloseableThreadLocal<TermVectorsReader>() {
     @Override
     protected TermVectorsReader initialValue() {
-      return (termVectorsReaderOrig == null) ?
-        null : termVectorsReaderOrig.clone();
+      return (termVectorsReaderOrig == null) ? null : termVectorsReaderOrig.clone();
     }
   };
-  
+
+  final CloseableThreadLocal<Map<String,Object>> docValuesLocal = new CloseableThreadLocal<Map<String,Object>>() {
+    @Override
+    protected Map<String,Object> initialValue() {
+      return new HashMap<String,Object>();
+    }
+  };
+
+  final CloseableThreadLocal<Map<String,Object>> normsLocal = new CloseableThreadLocal<Map<String,Object>>() {
+    @Override
+    protected Map<String,Object> initialValue() {
+      return new HashMap<String,Object>();
+    }
+  };
+
   private final Set<CoreClosedListener> coreClosedListeners = 
       Collections.synchronizedSet(new LinkedHashSet<CoreClosedListener>());
   
@@ -109,8 +129,20 @@ final class SegmentCoreReaders {
       // ask codec for its Norms: 
       // TODO: since we don't write any norms file if there are no norms,
       // 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);
+
+      if (fieldInfos.hasDocValues()) {
+        dvProducer = codec.docValuesFormat().fieldsProducer(segmentReadState);
+        assert dvProducer != null;
+      } else {
+        dvProducer = null;
+      }
+
+      if (fieldInfos.hasNorms()) {
+        normsProducer = codec.normsFormat().normsProducer(segmentReadState);
+        assert normsProducer != null;
+      } else {
+        normsProducer = null;
+      }
   
       fieldsReaderOrig = si.info.getCodec().storedFieldsFormat().fieldsReader(cfsDir, si.info, fieldInfos, context);
 
@@ -137,17 +169,123 @@ final class SegmentCoreReaders {
   void incRef() {
     ref.incrementAndGet();
   }
-  
+
+  NumericDocValues getNumericDocValues(String field) throws IOException {
+    FieldInfo fi = fieldInfos.fieldInfo(field);
+    if (fi == null) {
+      // Field does not exist
+      return null;
+    }
+    if (fi.getDocValuesType() == null) {
+      // Field was not indexed with doc values
+      return null;
+    }
+    if (fi.getDocValuesType() != DocValuesType.NUMERIC) {
+      // DocValues were not numeric
+      return null;
+    }
+
+    assert dvProducer != null;
+
+    Map<String,Object> dvFields = docValuesLocal.get();
+
+    NumericDocValues dvs = (NumericDocValues) dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getNumeric(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
+  }
+
+  BinaryDocValues getBinaryDocValues(String field) throws IOException {
+    FieldInfo fi = fieldInfos.fieldInfo(field);
+    if (fi == null) {
+      // Field does not exist
+      return null;
+    }
+    if (fi.getDocValuesType() == null) {
+      // Field was not indexed with doc values
+      return null;
+    }
+    if (fi.getDocValuesType() != DocValuesType.BINARY) {
+      // DocValues were not binary
+      return null;
+    }
+
+    assert dvProducer != null;
+
+    Map<String,Object> dvFields = docValuesLocal.get();
+
+    BinaryDocValues dvs = (BinaryDocValues) dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getBinary(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
+  }
+
+  SortedDocValues getSortedDocValues(String field) throws IOException {
+    FieldInfo fi = fieldInfos.fieldInfo(field);
+    if (fi == null) {
+      // Field does not exist
+      return null;
+    }
+    if (fi.getDocValuesType() == null) {
+      // Field was not indexed with doc values
+      return null;
+    }
+    if (fi.getDocValuesType() != DocValuesType.SORTED) {
+      // DocValues were not sorted
+      return null;
+    }
+
+    assert dvProducer != null;
+
+    Map<String,Object> dvFields = docValuesLocal.get();
+
+    SortedDocValues dvs = (SortedDocValues) dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getSorted(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
+  }
+
+  NumericDocValues getNormValues(String field) throws IOException {
+    FieldInfo fi = fieldInfos.fieldInfo(field);
+    if (fi == null) {
+      // Field does not exist
+      return null;
+    }
+    if (!fi.hasNorms()) {
+      return null;
+    }
+   
+    assert normsProducer != null;
+
+    Map<String,Object> normFields = normsLocal.get();
+
+    NumericDocValues norms = (NumericDocValues) normFields.get(field);
+    if (norms == null) {
+      norms = normsProducer.getNumeric(fi);
+      normFields.put(field, norms);
+    }
+
+    return norms;
+  }
+
   void decRef() throws IOException {
-    //System.out.println("core.decRef seg=" + owner.getSegmentInfo() + " rc=" + ref);
     if (ref.decrementAndGet() == 0) {
-      IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, perDocProducer,
-        termVectorsReaderOrig, fieldsReaderOrig, cfsReader, norms);
+      IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, fields, dvProducer,
+                    termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
       notifyCoreClosedListeners();
     }
   }
   
-  private final void notifyCoreClosedListeners() {
+  private void notifyCoreClosedListeners() {
     synchronized(coreClosedListeners) {
       for (CoreClosedListener listener : coreClosedListeners) {
         listener.onClose(owner);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Fri Feb  8 03:26:14 2013
@@ -19,16 +19,15 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.IOUtils;
@@ -70,14 +69,14 @@ final class SegmentMerger {
   /**
    * Add an IndexReader to the collection of readers that are to be merged
    */
-  final void add(IndexReader reader) {
+  void add(IndexReader reader) {
     for (final AtomicReaderContext ctx : reader.leaves()) {
       final AtomicReader r = ctx.reader();
       mergeState.readers.add(r);
     }
   }
 
-  final void add(SegmentReader reader) {
+  void add(SegmentReader reader) {
     mergeState.readers.add(reader);
   }
 
@@ -87,7 +86,7 @@ final class SegmentMerger {
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  final MergeState merge() throws IOException {
+  MergeState merge() throws IOException {
     // NOTE: it's important to add calls to
     // checkAbort.work(...) if you make any changes to this
     // method that will spend alot of time.  The frequency
@@ -96,7 +95,7 @@ final class SegmentMerger {
     // threads.
     
     mergeState.segmentInfo.setDocCount(setDocMaps());
-    mergeDocValuesAndNormsFieldInfos();
+    mergeFieldInfos();
     setMatchingSegmentReaders();
     long t0 = 0;
     if (mergeState.infoStream.isEnabled("SM")) {
@@ -123,7 +122,9 @@ final class SegmentMerger {
     if (mergeState.infoStream.isEnabled("SM")) {
       t0 = System.nanoTime();
     }
-    mergePerDoc(segmentWriteState);
+    if (mergeState.fieldInfos.hasDocValues()) {
+      mergeDocValues(segmentWriteState);
+    }
     if (mergeState.infoStream.isEnabled("SM")) {
       long t1 = System.nanoTime();
       mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge doc values [" + numMerged + " docs]");
@@ -159,6 +160,90 @@ final class SegmentMerger {
     return mergeState;
   }
 
+  private void mergeDocValues(SegmentWriteState segmentWriteState) throws IOException {
+
+    if (codec.docValuesFormat() != null) {
+      DocValuesConsumer consumer = codec.docValuesFormat().fieldsConsumer(segmentWriteState);
+      boolean success = false;
+      try {
+        for (FieldInfo field : mergeState.fieldInfos) {
+          DocValuesType type = field.getDocValuesType();
+          if (type != null) {
+            if (type == DocValuesType.NUMERIC) {
+              List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
+              for (AtomicReader reader : mergeState.readers) {
+                NumericDocValues values = reader.getNumericDocValues(field.name);
+                if (values == null) {
+                  values = NumericDocValues.EMPTY;
+                }
+                toMerge.add(values);
+              }
+              consumer.mergeNumericField(field, mergeState, toMerge);
+            } else if (type == DocValuesType.BINARY) {
+              List<BinaryDocValues> toMerge = new ArrayList<BinaryDocValues>();
+              for (AtomicReader reader : mergeState.readers) {
+                BinaryDocValues values = reader.getBinaryDocValues(field.name);
+                if (values == null) {
+                  values = BinaryDocValues.EMPTY;
+                }
+                toMerge.add(values);
+              }
+              consumer.mergeBinaryField(field, mergeState, toMerge);
+            } else if (type == DocValuesType.SORTED) {
+              List<SortedDocValues> toMerge = new ArrayList<SortedDocValues>();
+              for (AtomicReader reader : mergeState.readers) {
+                SortedDocValues values = reader.getSortedDocValues(field.name);
+                if (values == null) {
+                  values = SortedDocValues.EMPTY;
+                }
+                toMerge.add(values);
+              }
+              consumer.mergeSortedField(field, mergeState, toMerge);
+            } else {
+              throw new AssertionError("type=" + type);
+            }
+          }
+        }
+        success = true;
+      } finally {
+        if (success) {
+          IOUtils.close(consumer);
+        } else {
+          IOUtils.closeWhileHandlingException(consumer);            
+        }
+      }
+    }
+  }
+
+  private void mergeNorms(SegmentWriteState segmentWriteState) throws IOException {
+    if (codec.normsFormat() != null) {
+      DocValuesConsumer consumer = codec.normsFormat().normsConsumer(segmentWriteState);
+      boolean success = false;
+      try {
+        for (FieldInfo field : mergeState.fieldInfos) {
+          if (field.hasNorms()) {
+            List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
+            for (AtomicReader reader : mergeState.readers) {
+              NumericDocValues norms = reader.getNormValues(field.name);
+              if (norms == null) {
+                norms = NumericDocValues.EMPTY;
+              }
+              toMerge.add(norms);
+            }
+            consumer.mergeNumericField(field, mergeState, toMerge);
+          }
+        }
+        success = true;
+      } finally {
+        if (success) {
+          IOUtils.close(consumer);
+        } else {
+          IOUtils.closeWhileHandlingException(consumer);            
+        }
+      }
+    }
+  }
+
   private void setMatchingSegmentReaders() {
     // If the i'th reader is a SegmentReader and has
     // identical fieldName -> number mapping, then this
@@ -203,72 +288,15 @@ final class SegmentMerger {
     }
   }
   
-  // returns an updated typepromoter (tracking type and size) given a previous one,
-  // and a newly encountered docvalues
-  private TypePromoter mergeDocValuesType(TypePromoter previous, DocValues docValues) {
-    TypePromoter incoming = TypePromoter.create(docValues.getType(),  docValues.getValueSize());
-    if (previous == null) {
-      previous = TypePromoter.getIdentityPromoter();
-    }
-    return previous.promote(incoming);
-  }
-
-  // NOTE: this is actually merging all the fieldinfos
-  public void mergeDocValuesAndNormsFieldInfos() throws IOException {
-    // mapping from all docvalues fields found to their promoted types
-    // this is because FieldInfos does not store the
-    // valueSize
-    Map<FieldInfo,TypePromoter> docValuesTypes = new HashMap<FieldInfo,TypePromoter>();
-    Map<FieldInfo,TypePromoter> normValuesTypes = new HashMap<FieldInfo,TypePromoter>();
-
+  public void mergeFieldInfos() throws IOException {
     for (AtomicReader reader : mergeState.readers) {
       FieldInfos readerFieldInfos = reader.getFieldInfos();
       for (FieldInfo fi : readerFieldInfos) {
-        FieldInfo merged = fieldInfosBuilder.add(fi);
-        // update the type promotion mapping for this reader
-        if (fi.hasDocValues()) {
-          TypePromoter previous = docValuesTypes.get(merged);
-          docValuesTypes.put(merged, mergeDocValuesType(previous, reader.docValues(fi.name))); 
-        }
-        if (fi.hasNorms()) {
-          TypePromoter previous = normValuesTypes.get(merged);
-          normValuesTypes.put(merged, mergeDocValuesType(previous, reader.normValues(fi.name))); 
-        }
+        fieldInfosBuilder.add(fi);
       }
     }
-    updatePromoted(normValuesTypes, true);
-    updatePromoted(docValuesTypes, false);
     mergeState.fieldInfos = fieldInfosBuilder.finish();
   }
-  
-  protected void updatePromoted(Map<FieldInfo,TypePromoter> infoAndPromoter, boolean norms) {
-    // update any promoted doc values types:
-    for (Map.Entry<FieldInfo,TypePromoter> e : infoAndPromoter.entrySet()) {
-      FieldInfo fi = e.getKey();
-      TypePromoter promoter = e.getValue();
-      if (promoter == null) {
-        if (norms) {
-          fi.setNormValueType(null);
-        } else {
-          fi.setDocValuesType(null);
-        }
-      } else {
-        assert promoter != TypePromoter.getIdentityPromoter();
-        if (norms) {
-          if (fi.getNormType() != promoter.type() && !fi.omitsNorms()) {
-            // reset the type if we got promoted
-            fi.setNormValueType(promoter.type());
-          }  
-        } else {
-          if (fi.getDocValuesType() != promoter.type()) {
-            // reset the type if we got promoted
-            fi.setDocValuesType(promoter.type());
-          }
-        }
-      }
-    }
-  }
-
 
   /**
    *
@@ -290,7 +318,7 @@ final class SegmentMerger {
    * Merge the TermVectors from each of the segments into the new one.
    * @throws IOException if there is a low-level IO error
    */
-  private final int mergeVectors() throws IOException {
+  private int mergeVectors() throws IOException {
     final TermVectorsWriter termVectorsWriter = codec.termVectorsFormat().vectorsWriter(directory, mergeState.segmentInfo, context);
     
     try {
@@ -326,7 +354,7 @@ final class SegmentMerger {
     return docBase;
   }
 
-  private final void mergeTerms(SegmentWriteState segmentWriteState) throws IOException {
+  private void mergeTerms(SegmentWriteState segmentWriteState) throws IOException {
     
     final List<Fields> fields = new ArrayList<Fields>();
     final List<ReaderSlice> slices = new ArrayList<ReaderSlice>();
@@ -359,46 +387,4 @@ final class SegmentMerger {
       }
     }
   }
-
-  private void mergePerDoc(SegmentWriteState segmentWriteState) throws IOException {
-      final PerDocConsumer docsConsumer = codec.docValuesFormat()
-          .docsConsumer(new PerDocWriteState(segmentWriteState));
-      // TODO: remove this check when 3.x indexes are no longer supported
-      // (3.x indexes don't have docvalues)
-      if (docsConsumer == null) {
-        return;
-      }
-      boolean success = false;
-      try {
-        docsConsumer.merge(mergeState);
-        success = true;
-      } finally {
-        if (success) {
-          IOUtils.close(docsConsumer);
-        } else {
-          IOUtils.closeWhileHandlingException(docsConsumer);
-        }
-      }
-  }
-  
-  private void mergeNorms(SegmentWriteState segmentWriteState) throws IOException {
-    final PerDocConsumer docsConsumer = codec.normsFormat()
-        .docsConsumer(new PerDocWriteState(segmentWriteState));
-    // TODO: remove this check when 3.x indexes are no longer supported
-    // (3.x indexes don't have docvalues)
-    if (docsConsumer == null) {
-      return;
-    }
-    boolean success = false;
-    try {
-      docsConsumer.merge(mergeState);
-      success = true;
-    } finally {
-      if (success) {
-        IOUtils.close(docsConsumer);
-      } else {
-        IOUtils.closeWhileHandlingException(docsConsumer);
-      }
-    }
-  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentReadState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentReadState.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentReadState.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentReadState.java Fri Feb  8 03:26:14 2013
@@ -28,7 +28,7 @@ import org.apache.lucene.store.IOContext
  */
 public class SegmentReadState {
   /** {@link Directory} where this segment is read from. */ 
-  public final Directory dir;
+  public final Directory directory;
 
   /** {@link SegmentInfo} describing this segment. */
   public final SegmentInfo segmentInfo;
@@ -73,7 +73,7 @@ public class SegmentReadState {
                           IOContext context,
                           int termsIndexDivisor,
                           String segmentSuffix) {
-    this.dir = dir;
+    this.directory = dir;
     this.segmentInfo = info;
     this.fieldInfos = fieldInfos;
     this.context = context;
@@ -84,7 +84,7 @@ public class SegmentReadState {
   /** Create a {@code SegmentReadState}. */
   public SegmentReadState(SegmentReadState other,
                           String newSegmentSuffix) {
-    this.dir = other.dir;
+    this.directory = other.directory;
     this.segmentInfo = other.segmentInfo;
     this.fieldInfos = other.fieldInfos;
     this.context = other.context;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Fri Feb  8 03:26:14 2013
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.codecs.PerDocProducer;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.search.FieldCache; // javadocs
@@ -229,27 +228,30 @@ public final class SegmentReader extends
   public int getTermInfosIndexDivisor() {
     return core.termsIndexDivisor;
   }
-  
+
   @Override
-  public DocValues docValues(String field) throws IOException {
+  public NumericDocValues getNumericDocValues(String field) throws IOException {
     ensureOpen();
-    final PerDocProducer perDoc = core.perDocProducer;
-    if (perDoc == null) {
-      return null;
-    }
-    return perDoc.docValues(field);
+    return core.getNumericDocValues(field);
   }
-  
+
   @Override
-  public DocValues normValues(String field) throws IOException {
+  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
     ensureOpen();
-    final PerDocProducer perDoc = core.norms;
-    if (perDoc == null) {
-      return null;
-    }
-    return perDoc.docValues(field);
+    return core.getBinaryDocValues(field);
+  }
+
+  @Override
+  public SortedDocValues getSortedDocValues(String field) throws IOException {
+    ensureOpen();
+    return core.getSortedDocValues(field);
+  }
+
+  @Override
+  public NumericDocValues getNormValues(String field) throws IOException {
+    ensureOpen();
+    return core.getNormValues(field);
   }
-  
 
   /**
    * Called when the shared core for this SegmentReader

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Fri Feb  8 03:26:14 2013
@@ -24,6 +24,8 @@ import java.util.Map;
 import org.apache.lucene.util.Bits;
 
 import org.apache.lucene.index.DirectoryReader; // javadoc
+import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
+import org.apache.lucene.index.MultiDocValues.OrdinalMap;
 import org.apache.lucene.index.MultiReader; // javadoc
 
 /**
@@ -44,7 +46,6 @@ import org.apache.lucene.index.MultiRead
 public final class SlowCompositeReaderWrapper extends AtomicReader {
 
   private final CompositeReader in;
-  private final Map<String, DocValues> normsCache = new HashMap<String, DocValues>();
   private final Fields fields;
   private final Bits liveDocs;
   
@@ -83,25 +84,65 @@ public final class SlowCompositeReaderWr
   }
 
   @Override
-  public DocValues docValues(String field) throws IOException {
+  public NumericDocValues getNumericDocValues(String field) throws IOException {
     ensureOpen();
-    return MultiDocValues.getDocValues(in, field);
+    return MultiDocValues.getNumericValues(in, field);
   }
-  
+
+  @Override
+  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
+    ensureOpen();
+    return MultiDocValues.getBinaryValues(in, field);
+  }
+
   @Override
-  public synchronized DocValues normValues(String field) throws IOException {
+  public SortedDocValues getSortedDocValues(String field) throws IOException {
     ensureOpen();
-    DocValues values = normsCache.get(field);
-    if (values == null) {
-      values = MultiDocValues.getNormDocValues(in, field);
-      normsCache.put(field, values);
+    OrdinalMap map = null;
+    synchronized (cachedOrdMaps) {
+      map = cachedOrdMaps.get(field);
+      if (map == null) {
+        // uncached, or not a multi dv
+        SortedDocValues dv = MultiDocValues.getSortedValues(in, field);
+        if (dv instanceof MultiSortedDocValues) {
+          map = ((MultiSortedDocValues)dv).mapping;
+          if (map.owner == getCoreCacheKey()) {
+            cachedOrdMaps.put(field, map);
+          }
+        }
+        return dv;
+      }
     }
-    return values;
+    // cached multi dv
+    assert map != null;
+    int size = in.leaves().size();
+    final SortedDocValues[] values = new SortedDocValues[size];
+    final int[] starts = new int[size+1];
+    for (int i = 0; i < size; i++) {
+      AtomicReaderContext context = in.leaves().get(i);
+      SortedDocValues v = context.reader().getSortedDocValues(field);
+      if (v == null) {
+        v = SortedDocValues.EMPTY;
+      }
+      values[i] = v;
+      starts[i] = context.docBase;
+    }
+    starts[size] = maxDoc();
+    return new MultiSortedDocValues(values, starts, map);
+  }
+  
+  // TODO: this could really be a weak map somewhere else on the coreCacheKey,
+  // but do we really need to optimize slow-wrapper any more?
+  private final Map<String,OrdinalMap> cachedOrdMaps = new HashMap<String,OrdinalMap>();
+
+  @Override
+  public NumericDocValues getNormValues(String field) throws IOException {
+    ensureOpen();
+    return MultiDocValues.getNormValues(in, field);
   }
   
   @Override
-  public Fields getTermVectors(int docID)
-          throws IOException {
+  public Fields getTermVectors(int docID) throws IOException {
     ensureOpen();
     return in.getTermVectors(docID);
   }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java Fri Feb  8 03:26:14 2013
@@ -19,131 +19,10 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.StoredFieldsWriter;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
-
-/** This is a DocFieldConsumer that writes stored fields. */
-final class StoredFieldsConsumer {
-
-  StoredFieldsWriter fieldsWriter;
-  final DocumentsWriterPerThread docWriter;
-  int lastDocID;
-
-  int freeCount;
-
-  final DocumentsWriterPerThread.DocState docState;
-  final Codec codec;
-
-  public StoredFieldsConsumer(DocumentsWriterPerThread docWriter) {
-    this.docWriter = docWriter;
-    this.docState = docWriter.docState;
-    this.codec = docWriter.codec;
-  }
-
-  private int numStoredFields;
-  private IndexableField[] storedFields;
-  private FieldInfo[] fieldInfos;
-
-  public void reset() {
-    numStoredFields = 0;
-    storedFields = new IndexableField[1];
-    fieldInfos = new FieldInfo[1];
-  }
-
-  public void startDocument() {
-    reset();
-  }
-
-  public void flush(SegmentWriteState state) throws IOException {
-    int numDocs = state.segmentInfo.getDocCount();
-
-    if (numDocs > 0) {
-      // It's possible that all documents seen in this segment
-      // hit non-aborting exceptions, in which case we will
-      // not have yet init'd the FieldsWriter:
-      initFieldsWriter(state.context);
-      fill(numDocs);
-    }
-
-    if (fieldsWriter != null) {
-      try {
-        fieldsWriter.finish(state.fieldInfos, numDocs);
-      } finally {
-        fieldsWriter.close();
-        fieldsWriter = null;
-        lastDocID = 0;
-      }
-    }
-  }
-
-  private synchronized void initFieldsWriter(IOContext context) throws IOException {
-    if (fieldsWriter == null) {
-      fieldsWriter = codec.storedFieldsFormat().fieldsWriter(docWriter.directory, docWriter.getSegmentInfo(), context);
-      lastDocID = 0;
-    }
-  }
-
-  int allocCount;
-
-  void abort() {
-    reset();
-
-    if (fieldsWriter != null) {
-      fieldsWriter.abort();
-      fieldsWriter = null;
-      lastDocID = 0;
-    }
-  }
-
-  /** Fills in any hole in the docIDs */
-  void fill(int docID) throws IOException {
-    // We must "catch up" for all docs before us
-    // that had no stored fields:
-    while(lastDocID < docID) {
-      fieldsWriter.startDocument(0);
-      lastDocID++;
-      fieldsWriter.finishDocument();
-    }
-  }
-
-  void finishDocument() throws IOException {
-    assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument start");
-
-    initFieldsWriter(IOContext.DEFAULT);
-    fill(docState.docID);
-
-    if (fieldsWriter != null && numStoredFields > 0) {
-      fieldsWriter.startDocument(numStoredFields);
-      for (int i = 0; i < numStoredFields; i++) {
-        fieldsWriter.writeField(fieldInfos[i], storedFields[i]);
-      }
-      fieldsWriter.finishDocument();
-      lastDocID++;
-    }
-
-    reset();
-    assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
-  }
-
-  public void addField(IndexableField field, FieldInfo fieldInfo) {
-    if (numStoredFields == storedFields.length) {
-      int newSize = ArrayUtil.oversize(numStoredFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
-      IndexableField[] newArray = new IndexableField[newSize];
-      System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
-      storedFields = newArray;
-      
-      FieldInfo[] newInfoArray = new FieldInfo[newSize];
-      System.arraycopy(fieldInfos, 0, newInfoArray, 0, numStoredFields);
-      fieldInfos = newInfoArray;
-    }
-
-    storedFields[numStoredFields] = field;
-    fieldInfos[numStoredFields] = fieldInfo;
-    numStoredFields++;
-
-    assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField");
-  }
+abstract class StoredFieldsConsumer {
+  abstract void addField(int docID, IndexableField field, FieldInfo fieldInfo) throws IOException;
+  abstract void flush(SegmentWriteState state) throws IOException;
+  abstract void abort() throws IOException;
+  abstract void startDocument() throws IOException;
+  abstract void finishDocument() throws IOException;
 }

Copied: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java (from r1443717, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java&p1=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java&r1=1443717&r2=1443834&rev=1443834&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java Fri Feb  8 03:26:14 2013
@@ -44,12 +44,12 @@ final class StoredFieldsProcessor extend
   }
 
   private int numStoredFields;
-  private StorableField[] storedFields;
+  private IndexableField[] storedFields;
   private FieldInfo[] fieldInfos;
 
   public void reset() {
     numStoredFields = 0;
-    storedFields = new StorableField[1];
+    storedFields = new IndexableField[1];
     fieldInfos = new FieldInfo[1];
   }
   
@@ -133,11 +133,11 @@ final class StoredFieldsProcessor extend
   }
 
   @Override
-  public void addField(int docID, StorableField field, FieldInfo fieldInfo) {
+  public void addField(int docID, IndexableField field, FieldInfo fieldInfo) {
     if (field.fieldType().stored()) {
       if (numStoredFields == storedFields.length) {
         int newSize = ArrayUtil.oversize(numStoredFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
-        StorableField[] newArray = new StorableField[newSize];
+        IndexableField[] newArray = new IndexableField[newSize];
         System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
         storedFields = newArray;
       

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java Fri Feb  8 03:26:14 2013
@@ -188,7 +188,6 @@ final class TermVectorsConsumerPerField 
 
     termsHashPerField.reset();
 
-    // commit the termVectors once successful - FI will otherwise reset them
     fieldInfo.setStoreTermVectors();
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TermsHash.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TermsHash.java Fri Feb  8 03:26:14 2013
@@ -126,11 +126,6 @@ final class TermsHash extends InvertedDo
   }
 
   @Override
-  public boolean freeRAM() {
-    return false;
-  }
-
-  @Override
   void finishDocument() throws IOException {
     consumer.finishDocument(this);
     if (nextTermsHash != null) {

Copied: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TwoStoredFieldsConsumers.java (from r1443717, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TwoStoredFieldsConsumers.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TwoStoredFieldsConsumers.java?p2=lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TwoStoredFieldsConsumers.java&p1=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TwoStoredFieldsConsumers.java&r1=1443717&r2=1443834&rev=1443834&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TwoStoredFieldsConsumers.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/TwoStoredFieldsConsumers.java Fri Feb  8 03:26:14 2013
@@ -31,7 +31,7 @@ class TwoStoredFieldsConsumers extends S
   }
 
   @Override
-  public void addField(int docID, StorableField field, FieldInfo fieldInfo) throws IOException {
+  public void addField(int docID, IndexableField field, FieldInfo fieldInfo) throws IOException {
     first.addField(docID, field, fieldInfo);
     second.addField(docID, field, fieldInfo);
   }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/package.html?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/package.html (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/package.html Fri Feb  8 03:26:14 2013
@@ -257,7 +257,7 @@ its {@link org.apache.lucene.search.simi
 </p>
 <p>
 Additional user-supplied statistics can be added to the document as DocValues fields and
-accessed via {@link org.apache.lucene.index.AtomicReader#docValues}.
+accessed via {@link org.apache.lucene.index.AtomicReader#getNumericDocValues}.
 </p>
 <p>
 </body>

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/search/FieldCache.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/search/FieldCache.java Fri Feb  8 03:26:14 2013
@@ -21,19 +21,20 @@ import java.io.IOException;
 import java.io.PrintStream;
 
 import org.apache.lucene.analysis.NumericTokenStream; // for javadocs
-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.index.DocTermOrds;
 import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocTermOrds;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * Expert: Maintains caches of term values.
@@ -42,9 +43,47 @@ import org.apache.lucene.util.packed.Pac
  *
  * @since   lucene 1.4
  * @see org.apache.lucene.util.FieldCacheSanityChecker
+ *
+ * @lucene.internal
  */
 public interface FieldCache {
 
+  /** Field values as 8-bit signed bytes */
+  public static abstract class Bytes {
+    /** Return a single Byte representation of this field's value. */
+    public abstract byte get(int docID);
+  }
+
+  /** Field values as 16-bit signed shorts */
+  public static abstract class Shorts {
+    /** Return a short representation of this field's value. */
+    public abstract short get(int docID);
+  }
+
+  /** Field values as 32-bit signed integers */
+  public static abstract class Ints {
+    /** Return an integer representation of this field's value. */
+    public abstract int get(int docID);
+  }
+
+  /** Field values as 32-bit signed long integers */
+  public static abstract class Longs {
+    /** Return an long representation of this field's value. */
+    public abstract long get(int docID);
+  }
+
+  /** Field values as 32-bit floats */
+  public static abstract class Floats {
+    /** Return an float representation of this field's value. */
+    public abstract float get(int docID);
+  }
+
+  /** Field values as 64-bit doubles */
+  public static abstract class Doubles {
+    /** Return an double representation of this field's value. */
+    public abstract double get(int docID);
+  }
+
   /**
    * Placeholder indicating creation of this cache is currently in-progress.
    */
@@ -114,7 +153,7 @@ public interface FieldCache {
    * @see FieldCache#getDoubles(AtomicReader, String, FieldCache.DoubleParser, boolean)
    */
   public interface DoubleParser extends Parser {
-    /** Return an long representation of this field's value. */
+    /** Return an double representation of this field's value. */
     public double parseDouble(BytesRef term);
   }
 
@@ -333,12 +372,13 @@ public interface FieldCache {
   
  
   /** Checks the internal cache for an appropriate entry, and if none is found,
-   * reads the terms in <code>field</code> and returns a bit set at the size of
-   * <code>reader.maxDoc()</code>, with turned on bits for each docid that 
-   * does have a value for this field.
+   *  reads the terms in <code>field</code> and returns a bit set at the size of
+   *  <code>reader.maxDoc()</code>, with turned on bits for each docid that 
+   *  does have a value for this field.  Note that if the field was only indexed
+   *  as DocValues then this method will not work (it will return a Bits stating
+   *  that no documents contain the field).
    */
-  public Bits getDocsWithField(AtomicReader reader, String field) 
-  throws IOException;
+  public Bits getDocsWithField(AtomicReader reader, String field) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is
    * found, reads the terms in <code>field</code> as a single byte and returns an array
@@ -351,8 +391,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
@@ -366,8 +405,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
@@ -380,8 +418,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
@@ -395,8 +432,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
@@ -409,8 +445,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
@@ -424,8 +459,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
@@ -438,8 +472,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
@@ -453,8 +486,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
@@ -469,7 +501,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;
 
   /**
@@ -486,7 +518,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;
 
   /**
@@ -502,7 +534,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;
 
   /**
@@ -519,35 +551,18 @@ 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)
-          throws IOException;
-
-  /** Returned by {@link #getTerms} */
-  public abstract static class DocTerms {
-    /** The BytesRef argument must not be null; the method
-     *  returns the same BytesRef, or an empty (length=0)
-     *  BytesRef if the doc did not have this field or was
-     *  deleted. */
-    public abstract BytesRef getTerm(int docID, BytesRef ret);
-
-    /** Returns true if this doc has this field and is not
-     *  deleted. */
-    public abstract boolean exists(int docID);
-
-    /** Number of documents */
-    public abstract int size();
-  }
+  public Doubles getDoubles(AtomicReader reader, String field, DoubleParser parser, boolean setDocsWithField) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none
    * is found, reads the term values in <code>field</code>
-   * and returns a {@link DocTerms} instance, providing a
+   * and returns a {@link BinaryDocValues} instance, providing a
    * method to retrieve the term (as a BytesRef) per document.
    * @param reader  Used to get field values.
    * @param field   Which field contains the strings.
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public DocTerms getTerms (AtomicReader reader, String field)
+  public BinaryDocValues getTerms (AtomicReader reader, String field)
   throws IOException;
 
   /** Expert: just like {@link #getTerms(AtomicReader,String)},
@@ -555,76 +570,19 @@ public interface FieldCache {
    *  faster lookups (default is "true").  Note that the
    *  first call for a given reader and field "wins",
    *  subsequent calls will share the same cache entry. */
-  public DocTerms getTerms (AtomicReader reader, String field, float acceptableOverheadRatio)
-  throws IOException;
-
-  /** Returned by {@link #getTermsIndex} */
-  public abstract static class DocTermsIndex {
-
-    public int binarySearchLookup(BytesRef key, BytesRef spare) {
-      // this special case is the reason that Arrays.binarySearch() isn't useful.
-      if (key == null)
-        return 0;
-  
-      int low = 1;
-      int high = numOrd()-1;
-
-      while (low <= high) {
-        int mid = (low + high) >>> 1;
-        int cmp = lookup(mid, spare).compareTo(key);
-
-        if (cmp < 0)
-          low = mid + 1;
-        else if (cmp > 0)
-          high = mid - 1;
-        else
-          return mid; // key found
-      }
-      return -(low + 1);  // key not found.
-    }
-
-    /** The BytesRef argument must not be null; the method
-     *  returns the same BytesRef, or an empty (length=0)
-     *  BytesRef if this ord is the null ord (0). */
-    public abstract BytesRef lookup(int ord, BytesRef reuse);
-
-    /** Convenience method, to lookup the Term for a doc.
-     *  If this doc is deleted or did not have this field,
-     *  this will return an empty (length=0) BytesRef. */
-    public BytesRef getTerm(int docID, BytesRef reuse) {
-      return lookup(getOrd(docID), reuse);
-    }
-
-    /** Returns sort ord for this document.  Ord 0 is
-     *  reserved for docs that are deleted or did not have
-     *  this field.  */
-    public abstract int getOrd(int docID);
-
-    /** Returns total unique ord count; this includes +1 for
-     *  the null ord (always 0). */
-    public abstract int numOrd();
-
-    /** Number of documents */
-    public abstract int size();
-
-    /** Returns a TermsEnum that can iterate over the values in this index entry */
-    public abstract TermsEnum getTermsEnum();
-
-    /** @lucene.internal */
-    public abstract PackedInts.Reader getDocToOrd();
-  }
+  public BinaryDocValues getTerms (AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none
    * is found, reads the term values in <code>field</code>
-   * and returns a {@link DocTerms} instance, providing a
-   * method to retrieve the term (as a BytesRef) per document.
+   * and returns a {@link SortedDocValues} instance,
+   * providing methods to retrieve sort ordinals and terms
+   * (as a ByteRef) per document.
    * @param reader  Used to get field values.
    * @param field   Which field contains the strings.
    * @return The values in the given field for each document.
    * @throws IOException  If any error occurs.
    */
-  public DocTermsIndex getTermsIndex (AtomicReader reader, String field)
-  throws IOException;
+  public SortedDocValues getTermsIndex (AtomicReader reader, String field) throws IOException;
 
   /** Expert: just like {@link
    *  #getTermsIndex(AtomicReader,String)}, but you can specify
@@ -632,8 +590,7 @@ public interface FieldCache {
    *  faster lookups (default is "true").  Note that the
    *  first call for a given reader and field "wins",
    *  subsequent calls will share the same cache entry. */
-  public DocTermsIndex getTermsIndex (AtomicReader reader, String field, float acceptableOverheadRatio)
-  throws IOException;
+  public SortedDocValues getTermsIndex (AtomicReader reader, String field, float acceptableOverheadRatio) throws IOException;
 
   /**
    * Checks the internal cache for an appropriate entry, and if none is found, reads the term values
@@ -652,15 +609,44 @@ public interface FieldCache {
    * Can be useful for logging/debugging.
    * @lucene.experimental
    */
-  public static abstract class CacheEntry {
-    public abstract Object getReaderKey();
-    public abstract String getFieldName();
-    public abstract Class<?> getCacheType();
-    public abstract Object getCustom();
-    public abstract Object getValue();
-    private String size = null;
-    protected final void setEstimatedSize(String size) {
-      this.size = size;
+  public final class CacheEntry {
+
+    private final Object readerKey;
+    private final String fieldName;
+    private final Class<?> cacheType;
+    private final Object custom;
+    private final Object value;
+    private String size;
+
+    public CacheEntry(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;
+    }
+
+    public Object getReaderKey() {
+      return readerKey;
+    }
+
+    public String getFieldName() {
+      return fieldName;
+    }
+
+    public Class<?> getCacheType() {
+      return cacheType;
+    }
+
+    public Object getCustom() {
+      return custom;
+    }
+
+    public Object getValue() {
+      return value;
     }
 
     /** 
@@ -668,19 +654,18 @@ public interface FieldCache {
      * @see #getEstimatedSize
      */
     public void estimateSize() {
-      long size = RamUsageEstimator.sizeOf(getValue());
-      setEstimatedSize(RamUsageEstimator.humanReadableUnits(size));
+      long bytesUsed = RamUsageEstimator.sizeOf(getValue());
+      size = RamUsageEstimator.humanReadableUnits(bytesUsed);
     }
 
     /**
      * The most recently estimated size of the value, null unless 
      * estimateSize has been called.
      */
-    public final String getEstimatedSize() {
+    public String getEstimatedSize() {
       return size;
     }
     
-    
     @Override
     public String toString() {
       StringBuilder b = new StringBuilder();
@@ -697,7 +682,6 @@ public interface FieldCache {
 
       return b.toString();
     }
-  
   }
   
   /**