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 2016/09/21 13:42:18 UTC

[14/16] lucene-solr:master: LUCENE-7407: switch doc values usage to an iterator API, based on DocIdSetIterator, instead of random acces, freeing codecs for future improvements

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
index c73e770..8be0f66 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene53/Lucene53NormsProducer.java
@@ -17,9 +17,6 @@
 package org.apache.lucene.codecs.lucene53;
 
 
-import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT;
-import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_START;
-
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -30,13 +27,19 @@ import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.LegacyNumericDocValues;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.LegacyNumericDocValuesWrapper;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RandomAccessInput;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.IOUtils;
 
+import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_CURRENT;
+import static org.apache.lucene.codecs.lucene53.Lucene53NormsFormat.VERSION_START;
+
 /**
  * Reader for {@link Lucene53NormsFormat}
  */
@@ -114,22 +117,23 @@ class Lucene53NormsProducer extends NormsProducer {
   public NumericDocValues getNorms(FieldInfo field) throws IOException {
     final NormsEntry entry = norms.get(field.number);
 
+    LegacyNumericDocValues norms;
+
     if (entry.bytesPerValue == 0) {
       final long value = entry.offset;
-      return new NumericDocValues() {
-        @Override
-        public long get(int docID) {
-          return value;
-        }
-      };
-    }
-
-    RandomAccessInput slice;
-    synchronized (data) {
-      switch (entry.bytesPerValue) {
+      norms = new LegacyNumericDocValues() {
+          @Override
+          public long get(int docID) {
+            return value;
+          }
+        };
+    } else {
+      RandomAccessInput slice;
+      synchronized (data) {
+        switch (entry.bytesPerValue) {
         case 1: 
           slice = data.randomAccessSlice(entry.offset, maxDoc);
-          return new NumericDocValues() {
+          norms = new LegacyNumericDocValues() {
             @Override
             public long get(int docID) {
               try {
@@ -139,9 +143,10 @@ class Lucene53NormsProducer extends NormsProducer {
               }
             }
           };
+          break;
         case 2: 
           slice = data.randomAccessSlice(entry.offset, maxDoc * 2L);
-          return new NumericDocValues() {
+          norms = new LegacyNumericDocValues() {
             @Override
             public long get(int docID) {
               try {
@@ -151,9 +156,10 @@ class Lucene53NormsProducer extends NormsProducer {
               }
             }
           };
+          break;
         case 4: 
           slice = data.randomAccessSlice(entry.offset, maxDoc * 4L);
-          return new NumericDocValues() {
+          norms = new LegacyNumericDocValues() {
             @Override
             public long get(int docID) {
               try {
@@ -163,9 +169,10 @@ class Lucene53NormsProducer extends NormsProducer {
               }
             }
           };
+          break;
         case 8: 
           slice = data.randomAccessSlice(entry.offset, maxDoc * 8L);
-          return new NumericDocValues() {
+          norms = new LegacyNumericDocValues() {
             @Override
             public long get(int docID) {
               try {
@@ -175,10 +182,14 @@ class Lucene53NormsProducer extends NormsProducer {
               }
             }
           };
+          break;
         default:
           throw new AssertionError();
+        }
       }
     }
+
+    return new LegacyNumericDocValuesWrapper(new Bits.MatchAllBits(maxDoc), norms);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
index 96acfd2..217cd43 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesConsumer.java
@@ -32,6 +32,8 @@ import java.util.stream.StreamSupport;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.LegacyDocValuesIterables;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
@@ -42,8 +44,8 @@ import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.MathUtil;
-import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
+import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.packed.DirectMonotonicWriter;
 import org.apache.lucene.util.packed.DirectWriter;
@@ -85,8 +87,8 @@ final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Close
   }
   
   @Override
-  public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
-    addNumericField(field, values, NumberType.VALUE);
+  public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), NumberType.VALUE);
   }
 
   void addNumericField(FieldInfo field, Iterable<Number> values, NumberType numberType) throws IOException {
@@ -351,7 +353,11 @@ final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Close
   }
 
   @Override
-  public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+  public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc));
+  }
+  
+  private void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
     // write the byte[] data
     meta.writeVInt(field.number);
     meta.writeByte(Lucene54DocValuesFormat.BINARY);
@@ -508,7 +514,6 @@ final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Close
       addReverseTermIndex(field, values, maxLength);
     }
   }
-  
   // writes term dictionary "block"
   // first term is absolute encoded as vint length + bytes.
   // lengths of subsequent N terms are encoded as either N bytes or N shorts.
@@ -572,21 +577,32 @@ final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Close
   }
 
   @Override
-  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+  public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(Lucene54DocValuesFormat.SORTED);
+    addTermsDict(field, LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field)));
+    addNumericField(field, LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc), NumberType.ORDINAL);
+  }
+
+  private void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> ords) throws IOException {
     meta.writeVInt(field.number);
     meta.writeByte(Lucene54DocValuesFormat.SORTED);
     addTermsDict(field, values);
-    addNumericField(field, docToOrd, NumberType.ORDINAL);
+    addNumericField(field, ords, NumberType.ORDINAL);
   }
 
   @Override
-  public void addSortedNumericField(FieldInfo field, final Iterable<Number> docToValueCount, final Iterable<Number> values) throws IOException {
+  public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException {
+
+    final Iterable<Number> docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc);
+    final Iterable<Number> values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field);
+    
     meta.writeVInt(field.number);
     meta.writeByte(Lucene54DocValuesFormat.SORTED_NUMERIC);
     if (isSingleValued(docToValueCount)) {
       meta.writeVInt(SORTED_SINGLE_VALUED);
       // The field is single-valued, we can encode it as NUMERIC
-      addNumericField(field, singletonView(docToValueCount, values, null));
+      addNumericField(field, singletonView(docToValueCount, values, null), NumberType.VALUE);
     } else {
       final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
       if (uniqueValueSets != null) {
@@ -608,7 +624,12 @@ final class Lucene54DocValuesConsumer extends DocValuesConsumer implements Close
   }
 
   @Override
-  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+  public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+
+    Iterable<BytesRef> values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field));
+    Iterable<Number> docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc);
+    Iterable<Number> ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field);
+
     meta.writeVInt(field.number);
     meta.writeByte(Lucene54DocValuesFormat.SORTED_SET);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
index 67027ea..62e631c 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/Lucene54DocValuesProducer.java
@@ -29,20 +29,8 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.lucene54.Lucene54DocValuesConsumer.NumberType;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.*;
 import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.RandomAccessOrds;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RandomAccessInput;
@@ -452,7 +440,59 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
   @Override
   public NumericDocValues getNumeric(FieldInfo field) throws IOException {
     NumericEntry entry = numerics.get(field.name);
-    return getNumeric(entry);
+    Bits docsWithField;
+
+    if (entry.format == SPARSE_COMPRESSED) {
+      // TODO: make a real iterator in this case!
+      docsWithField = getSparseLiveBits(entry);
+    } else {
+      if (entry.missingOffset == ALL_MISSING) {
+        return DocValues.emptyNumeric();
+      } else if (entry.missingOffset == ALL_LIVE) {
+        LongValues values = getNumeric(entry);
+        return new NumericDocValues() {
+          private int docID = -1;
+
+          @Override
+          public int docID() {
+            return docID;
+          }
+
+          @Override
+          public int nextDoc() {
+            docID++;
+            if (docID == maxDoc) {
+              docID = NO_MORE_DOCS;
+            }
+            return docID;
+          }
+
+          @Override
+          public int advance(int target) {
+            if (target >= maxDoc) {
+              docID = NO_MORE_DOCS;
+            } else {
+              docID = target;
+            }
+            return docID;
+          }
+
+          @Override
+          public long cost() {
+            // TODO
+            return 0;
+          }
+
+          @Override
+          public long longValue() {
+            return values.get(docID);
+          }
+        };
+      } else {
+        docsWithField = getLiveBits(entry.missingOffset, maxDoc);
+      }
+    }
+    return new LegacyNumericDocValuesWrapper(docsWithField, getNumeric(entry));
   }
 
   @Override
@@ -545,7 +585,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
     }
   }
 
-  static class SparseBits implements Bits {
+  static final class SparseBits implements Bits {
 
     final long maxDoc, docIDsLength, firstDocId;
     final LongValues docIds;
@@ -681,8 +721,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
 
   }
 
-  @Override
-  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+  LegacyBinaryDocValues getLegacyBinary(FieldInfo field) throws IOException {
     BinaryEntry bytes = binaries.get(field.name);
     switch(bytes.format) {
       case BINARY_FIXED_UNCOMPRESSED:
@@ -696,7 +735,13 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
     }
   }
 
-  private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+  @Override
+  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BinaryEntry be = binaries.get(field.name);
+    return new LegacyBinaryDocValuesWrapper(getLiveBits(be.missingOffset, maxDoc), getLegacyBinary(field));
+  }
+
+  private LegacyBinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
     final IndexInput data = this.data.slice("fixed-binary", bytes.offset, bytes.count * bytes.maxLength);
 
     final BytesRef term = new BytesRef(bytes.maxLength);
@@ -717,7 +762,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
     };
   }
 
-  private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+  private LegacyBinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
     final RandomAccessInput addressesData = this.data.randomAccessSlice(bytes.addressesOffset, bytes.addressesEndOffset - bytes.addressesOffset);
     final LongValues addresses = DirectMonotonicReader.getInstance(bytes.addressesMeta, addressesData);
 
@@ -778,7 +823,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
     return index;
   }
 
-  private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+  private LegacyBinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
     final MonotonicBlockPackedReader addresses = getIntervalInstance(field, bytes);
     final ReverseTermsIndex index = getReverseIndexInstance(field, bytes);
     assert addresses.size() > 0; // we don't have to handle empty case
@@ -789,14 +834,55 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
   @Override
   public SortedDocValues getSorted(FieldInfo field) throws IOException {
     final int valueCount = (int) binaries.get(field.name).count;
-    final BinaryDocValues binary = getBinary(field);
+    final LegacyBinaryDocValues binary = getLegacyBinary(field);
     NumericEntry entry = ords.get(field.name);
     final LongValues ordinals = getNumeric(entry);
     return new SortedDocValues() {
+      private int docID = -1;
+      private int ord;
 
       @Override
-      public int getOrd(int docID) {
-        return (int) ordinals.get(docID);
+      public int docID() {
+        return docID;
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        assert docID != NO_MORE_DOCS;
+        while (true) {
+          docID++;
+          if (docID == maxDoc) {
+            docID = NO_MORE_DOCS;
+            break;
+          }
+          ord = (int) ordinals.get(docID);
+          if (ord != -1) {
+            break;
+          }
+        }
+        return docID;
+      }
+
+      @Override
+      public int advance(int target) throws IOException {
+        if (target >= maxDoc) {
+          docID = NO_MORE_DOCS;
+          return docID;
+        } else {
+          docID = target-1;
+          return nextDoc();
+        }
+      }
+          
+      @Override
+      public int ordValue() {
+        return ord;
+      }
+
+      @Override
+      public long cost() {
+        // TODO
+        return 0;
       }
 
       @Override
@@ -847,7 +933,58 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
       } else {
         docsWithField = getLiveBits(numericEntry.missingOffset, maxDoc);
       }
-      return DocValues.singleton(values, docsWithField);
+      return new SortedNumericDocValues() {
+        int docID = -1;
+
+        @Override
+        public int docID() {
+          return docID;
+        }
+
+        @Override
+        public int nextDoc() {
+          while (true) {
+            docID++;
+            if (docID == maxDoc) {
+              docID = NO_MORE_DOCS;
+              break;
+            }
+            
+            if (docsWithField.get(docID)) {
+              // TODO: use .nextSetBit here, at least!!
+              break;
+            }
+          }
+          return docID;
+        }
+
+        @Override
+        public int advance(int target) {
+          if (target >= maxDoc) {
+            docID = NO_MORE_DOCS;
+            return docID;
+          } else {
+            docID = target-1;
+            return nextDoc();
+          }
+        }
+
+        @Override
+        public long cost() {
+          // TODO
+          return 0;
+        }
+
+        @Override
+        public int docValueCount() {
+          return 1;
+        }
+
+        @Override
+        public long nextValue() {
+          return values.get(docID);
+        }
+      };
     } else if (ss.format == SORTED_WITH_ADDRESSES) {
       NumericEntry numericEntry = numerics.get(field.name);
       final LongValues values = getNumeric(numericEntry);
@@ -856,22 +993,58 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
       return new SortedNumericDocValues() {
         long startOffset;
         long endOffset;
+        int docID = -1;
+        long upto;
 
         @Override
-        public void setDocument(int doc) {
-          startOffset = ordIndex.get(doc);
-          endOffset = ordIndex.get(doc+1L);
+        public int docID() {
+          return docID;
         }
 
         @Override
-        public long valueAt(int index) {
-          return values.get(startOffset + index);
+        public int nextDoc() {
+          while (true) {
+            docID++;
+            if (docID == maxDoc) {
+              docID = NO_MORE_DOCS;
+              return docID;
+            }
+            startOffset = ordIndex.get(docID);
+            endOffset = ordIndex.get(docID+1L);
+            if (endOffset > startOffset) {
+              break;
+            }
+          }
+          upto = startOffset;
+          return docID;
         }
 
         @Override
-        public int count() {
+        public int advance(int target) {
+          if (target >= maxDoc) {
+            docID = NO_MORE_DOCS;
+            return docID;
+          } else {
+            docID = target-1;
+            return nextDoc();
+          }
+        }
+        
+        @Override
+        public long cost() {
+          // TODO
+          return 0;
+        }
+        
+        @Override
+        public int docValueCount() {
           return (int) (endOffset - startOffset);
         }
+        
+        @Override
+        public long nextValue() {
+          return values.get(upto++);
+        }
       };
     } else if (ss.format == SORTED_SET_TABLE) {
       NumericEntry entry = ords.get(field.name);
@@ -882,23 +1055,59 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
       return new SortedNumericDocValues() {
         int startOffset;
         int endOffset;
+        int docID = -1;
+        int upto;
 
         @Override
-        public void setDocument(int doc) {
-          final int ord = (int) ordinals.get(doc);
-          startOffset = offsets[ord];
-          endOffset = offsets[ord + 1];
+        public int docID() {
+          return docID;
         }
 
         @Override
-        public long valueAt(int index) {
-          return table[startOffset + index];
+        public int nextDoc() {
+          while (true) {
+            docID++;
+            if (docID == maxDoc) {
+              docID = NO_MORE_DOCS;
+              return docID;
+            }
+            int ord = (int) ordinals.get(docID);
+            startOffset = offsets[ord];
+            endOffset = offsets[ord+1];
+            if (endOffset > startOffset) {
+              break;
+            }
+          }
+          upto = startOffset;
+          return docID;
         }
 
         @Override
-        public int count() {
+        public int advance(int target) {
+          if (target >= maxDoc) {
+            docID = NO_MORE_DOCS;
+            return docID;
+          } else {
+            docID = target-1;
+            return nextDoc();
+          }
+        }
+        
+        @Override
+        public long cost() {
+          // TODO
+          return 0;
+        }
+
+        @Override
+        public int docValueCount() {
           return endOffset - startOffset;
         }
+        
+        @Override
+        public long nextValue() {
+          return table[upto++];
+        }
       };
     } else {
       throw new AssertionError();
@@ -910,8 +1119,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
     SortedSetEntry ss = sortedSets.get(field.name);
     switch (ss.format) {
       case SORTED_SINGLE_VALUED:
-        final SortedDocValues values = getSorted(field);
-        return DocValues.singleton(values);
+        return DocValues.singleton(getSorted(field));
       case SORTED_WITH_ADDRESSES:
         return getSortedSetWithAddresses(field);
       case SORTED_SET_TABLE:
@@ -924,12 +1132,12 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
   private SortedSetDocValues getSortedSetWithAddresses(FieldInfo field) throws IOException {
     final long valueCount = binaries.get(field.name).count;
     // we keep the byte[]s and list of ords on disk, these could be large
-    final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
+    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
     final LongValues ordinals = getNumeric(ords.get(field.name));
     // but the addresses to the ord stream are in RAM
     final LongValues ordIndex = getOrdIndexInstance(field, ordIndexes.get(field.name));
 
-    return new RandomAccessOrds() {
+    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
       long startOffset;
       long offset;
       long endOffset;
@@ -978,28 +1186,18 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
           return super.termsEnum();
         }
       }
-
-      @Override
-      public long ordAt(int index) {
-        return ordinals.get(startOffset + index);
-      }
-
-      @Override
-      public int cardinality() {
-        return (int) (endOffset - startOffset);
-      }
-    };
+      }, maxDoc);
   }
 
   private SortedSetDocValues getSortedSetTable(FieldInfo field, SortedSetEntry ss) throws IOException {
     final long valueCount = binaries.get(field.name).count;
-    final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
+    final LongBinaryDocValues binary = (LongBinaryDocValues) getLegacyBinary(field);
     final LongValues ordinals = getNumeric(ords.get(field.name));
 
     final long[] table = ss.table;
     final int[] offsets = ss.tableOffsets;
 
-    return new RandomAccessOrds() {
+    return new LegacySortedSetDocValuesWrapper(new LegacySortedSetDocValues() {
 
       int offset, startOffset, endOffset;
 
@@ -1011,11 +1209,6 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
       }
 
       @Override
-      public long ordAt(int index) {
-        return table[startOffset + index];
-      }
-
-      @Override
       public long nextOrd() {
         if (offset == endOffset) {
           return NO_MORE_ORDS;
@@ -1025,11 +1218,6 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
       }
 
       @Override
-      public int cardinality() {
-        return endOffset - startOffset;
-      }
-
-      @Override
       public BytesRef lookupOrd(long ord) {
         return binary.get(ord);
       }
@@ -1056,8 +1244,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
           return super.termsEnum();
         }
       }
-
-    };
+      }, maxDoc);
   }
 
   private Bits getLiveBits(final long offset, final int count) throws IOException {
@@ -1093,30 +1280,6 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
   }
 
   @Override
-  public Bits getDocsWithField(FieldInfo field) throws IOException {
-    switch(field.getDocValuesType()) {
-      case SORTED_SET:
-        return DocValues.docsWithValue(getSortedSet(field), maxDoc);
-      case SORTED_NUMERIC:
-        return DocValues.docsWithValue(getSortedNumeric(field), maxDoc);
-      case SORTED:
-        return DocValues.docsWithValue(getSorted(field), maxDoc);
-      case BINARY:
-        BinaryEntry be = binaries.get(field.name);
-        return getLiveBits(be.missingOffset, maxDoc);
-      case NUMERIC:
-        NumericEntry ne = numerics.get(field.name);
-        if (ne.format == SPARSE_COMPRESSED) {
-          return getSparseLiveBits(ne);
-        } else {
-          return getLiveBits(ne.missingOffset, maxDoc);
-        }
-      default:
-        throw new AssertionError();
-    }
-  }
-
-  @Override
   public synchronized DocValuesProducer getMergeInstance() throws IOException {
     return new Lucene54DocValuesProducer(this);
   }
@@ -1191,7 +1354,7 @@ final class Lucene54DocValuesProducer extends DocValuesProducer implements Close
   }
 
   // internally we compose complex dv (sorted/sortedset) from other ones
-  static abstract class LongBinaryDocValues extends BinaryDocValues {
+  static abstract class LongBinaryDocValues extends LegacyBinaryDocValues {
     @Override
     public final BytesRef get(int docID) {
       return get((long)docID);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
index 5dec06b..2c19222 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene54/package-info.java
@@ -366,7 +366,7 @@
  * <li>In version 4.0, the format of the inverted index became extensible via
  * the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
  * ({@code DocValues}) was introduced. Normalization factors need no longer be a 
- * single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. 
+ * single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
  * Terms need not be unicode strings, they can be any byte sequence. Term offsets 
  * can optionally be indexed into the postings lists. Payloads can be stored in the 
  * term vectors.</li>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
index baadf1e..6772cad 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
@@ -40,8 +40,6 @@ import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 /**
@@ -105,28 +103,28 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
     }
     
     @Override
-    public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
-      getInstance(field).addNumericField(field, values);
+    public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      getInstance(field).addNumericField(field, valuesProducer);
     }
 
     @Override
-    public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
-      getInstance(field).addBinaryField(field, values);
+    public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      getInstance(field).addBinaryField(field, valuesProducer);
     }
 
     @Override
-    public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
-      getInstance(field).addSortedField(field, values, docToOrd);
+    public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      getInstance(field).addSortedField(field, valuesProducer);
     }
 
     @Override
-    public void addSortedNumericField(FieldInfo field, Iterable<Number> docToValueCount, Iterable<Number> values) throws IOException {
-      getInstance(field).addSortedNumericField(field, docToValueCount, values);
+    public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      getInstance(field).addSortedNumericField(field, valuesProducer);
     }
 
     @Override
-    public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
-      getInstance(field).addSortedSetField(field, values, docToOrdCount, ords);
+    public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
+      getInstance(field).addSortedSetField(field, valuesProducer);
     }
 
     private DocValuesConsumer getInstance(FieldInfo field) throws IOException {
@@ -308,12 +306,6 @@ public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
     }
     
     @Override
-    public Bits getDocsWithField(FieldInfo field) throws IOException {
-      DocValuesProducer producer = fields.get(field.name);
-      return producer == null ? null : producer.getDocsWithField(field);
-    }
-
-    @Override
     public void close() throws IOException {
       IOUtils.close(formats.values());
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java
index b6a16de..94093b2 100644
--- a/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/BinaryDocValuesField.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.document;
 
-
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.util.BytesRef;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java
index d60a4e0..f9da94f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java
@@ -14,23 +14,24 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.index;
 
+package org.apache.lucene.index;
 
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.BytesRef;
 
 /**
- * A per-document byte[]
+ * A per-document numeric value.
  */
-public abstract class BinaryDocValues {
+public abstract class BinaryDocValues extends DocIdSetIterator {
   
   /** Sole constructor. (For invocation by subclass 
-   * constructors, typically implicit.) */
+   *  constructors, typically implicit.) */
   protected BinaryDocValues() {}
 
-  /** Lookup the value for document.  The returned {@link BytesRef} may be
-   * re-used across calls to {@link #get(int)} so make sure to
-   * {@link BytesRef#deepCopyOf(BytesRef) copy it} if you want to keep it
-   * around. */
-  public abstract BytesRef get(int docID);
+  /**
+   * Returns the numeric value for the current document ID.
+   * @return numeric value
+   */
+  public abstract BytesRef binaryValue();
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
index 03d9ff3..cbcfec5 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
@@ -18,8 +18,6 @@ package org.apache.lucene.index;
 
 
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.store.DataInput;
@@ -115,64 +113,76 @@ class BinaryDocValuesWriter extends DocValuesWriter {
     bytes.freeze(false);
     final PackedLongValues lengths = this.lengths.build();
     dvConsumer.addBinaryField(fieldInfo,
-                              new Iterable<BytesRef>() {
+                              new EmptyDocValuesProducer() {
                                 @Override
-                                public Iterator<BytesRef> iterator() {
-                                   return new BytesIterator(maxDoc, lengths);
+                                public BinaryDocValues getBinary(FieldInfo fieldInfoIn) {
+                                  if (fieldInfoIn != fieldInfo) {
+                                    throw new IllegalArgumentException("wrong fieldInfo");
+                                  }
+                                  return new BufferedBinaryDocValues(maxDoc, lengths);
                                 }
                               });
   }
 
   // iterates over the values we have in ram
-  private class BytesIterator implements Iterator<BytesRef> {
+  private class BufferedBinaryDocValues extends BinaryDocValues {
     final BytesRefBuilder value = new BytesRefBuilder();
     final PackedLongValues.Iterator lengthsIterator;
     final DataInput bytesIterator = bytes.getDataInput();
-    final int size = (int) lengths.size();
+    final int size;
     final int maxDoc;
-    int upto;
+    private int docID = -1;
     
-    BytesIterator(int maxDoc, PackedLongValues lengths) {
+    BufferedBinaryDocValues(int maxDoc, PackedLongValues lengths) {
       this.maxDoc = maxDoc;
       this.lengthsIterator = lengths.iterator();
+      this.size = (int) lengths.size();
     }
-    
+
     @Override
-    public boolean hasNext() {
-      return upto < maxDoc;
+    public int docID() {
+      return docID;
     }
 
     @Override
-    public BytesRef next() {
-      if (!hasNext()) {
-        throw new NoSuchElementException();
-      }
-      final BytesRef v;
-      if (upto < size) {
-        int length = (int) lengthsIterator.next();
-        value.grow(length);
-        value.setLength(length);
-        try {
-          bytesIterator.readBytes(value.bytes(), 0, value.length());
-        } catch (IOException ioe) {
-          // Should never happen!
-          throw new RuntimeException(ioe);
-        }
-        if (docsWithField.get(upto)) {
-          v = value.get();
+    public int nextDoc() throws IOException {
+      if (docID == size-1) {
+        docID = NO_MORE_DOCS;
+      } else {
+        int next = docsWithField.nextSetBit(docID+1);
+        if (next == NO_MORE_DOCS) {
+          docID = NO_MORE_DOCS;
         } else {
-          v = null;
+
+          int length = 0;
+
+          // skip missing values:
+          while (docID < next) {
+            docID++;
+            length = (int) lengthsIterator.next();
+            assert docID == next || length == 0;
+          }
+          value.grow(length);
+          value.setLength(length);
+          bytesIterator.readBytes(value.bytes(), 0, length);
         }
-      } else {
-        v = null;
       }
-      upto++;
-      return v;
+      return docID;
     }
 
     @Override
-    public void remove() {
+    public int advance(int target) {
       throw new UnsupportedOperationException();
     }
+
+    @Override
+    public long cost() {
+      return docsWithField.cardinality();
+    }
+
+    @Override
+    public BytesRef binaryValue() {
+      return value.get();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 9dee2d1..27d7c50 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -66,6 +66,8 @@ import org.apache.lucene.util.Version;
 import org.apache.lucene.util.automaton.Automata;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+
 /**
  * Basic tool and API to check the health of an index and
  * write a new segments file that removes reference to
@@ -999,7 +1001,7 @@ public final class CheckIndex implements Closeable {
       }
       for (FieldInfo info : reader.getFieldInfos()) {
         if (info.hasNorms()) {
-          checkNumericDocValues(info.name, reader.maxDoc(), normsReader.getNorms(info), new Bits.MatchAllBits(reader.maxDoc()));
+          checkNumericDocValues(info.name, normsReader.getNorms(info));
           ++status.totFields;
         }
       }
@@ -2056,34 +2058,36 @@ public final class CheckIndex implements Closeable {
     }
     return status;
   }
-  
-  private static void checkBinaryDocValues(String fieldName, int maxDoc, BinaryDocValues dv, Bits docsWithField) {
-    for (int i = 0; i < maxDoc; i++) {
-      final BytesRef term = dv.get(i);
-      assert term.isValid();
-      if (docsWithField.get(i) == false && term.length > 0) {
-        throw new RuntimeException("dv for field: " + fieldName + " is missing but has value=" + term + " for doc: " + i);
-      }
+
+  private static void checkBinaryDocValues(String fieldName, int maxDoc, BinaryDocValues bdv) throws IOException {
+    int doc;
+    if (bdv.docID() != -1) {
+      throw new RuntimeException("binary dv iterator for field: " + fieldName + " should start at docID=-1, but got " + bdv.docID());
+    }
+    // TODO: we could add stats to DVs, e.g. total doc count w/ a value for this field
+    // TODO: check advance too
+    while ((doc = bdv.nextDoc()) != NO_MORE_DOCS) {
+      BytesRef value = bdv.binaryValue();
+      value.isValid();
     }
   }
-  
-  private static void checkSortedDocValues(String fieldName, int maxDoc, SortedDocValues dv, Bits docsWithField) {
-    checkBinaryDocValues(fieldName, maxDoc, dv, docsWithField);
+
+  private static void checkSortedDocValues(String fieldName, int maxDoc, SortedDocValues dv) throws IOException {
+    if (dv.docID() != -1) {
+      throw new RuntimeException("sorted dv iterator for field: " + fieldName + " should start at docID=-1, but got " + dv.docID());
+    }
     final int maxOrd = dv.getValueCount()-1;
     FixedBitSet seenOrds = new FixedBitSet(dv.getValueCount());
     int maxOrd2 = -1;
-    for (int i = 0; i < maxDoc; i++) {
-      int ord = dv.getOrd(i);
+    int docID;
+    // TODO: check advance too
+    while ((docID = dv.nextDoc()) != NO_MORE_DOCS) {
+      int ord = dv.ordValue();
       if (ord == -1) {
-        if (docsWithField.get(i)) {
-          throw new RuntimeException("dv for field: " + fieldName + " has -1 ord but is not marked missing for doc: " + i);
-        }
+        throw new RuntimeException("dv for field: " + fieldName + " has -1 ord");
       } else if (ord < -1 || ord > maxOrd) {
         throw new RuntimeException("ord out of bounds: " + ord);
       } else {
-        if (!docsWithField.get(i)) {
-          throw new RuntimeException("dv for field: " + fieldName + " is missing but has ord=" + ord + " for doc: " + i);
-        }
         maxOrd2 = Math.max(maxOrd2, ord);
         seenOrds.set(ord);
       }
@@ -2097,7 +2101,7 @@ public final class CheckIndex implements Closeable {
     BytesRef lastValue = null;
     for (int i = 0; i <= maxOrd; i++) {
       final BytesRef term = dv.lookupOrd(i);
-      assert term.isValid();
+      term.isValid();
       if (lastValue != null) {
         if (term.compareTo(lastValue) <= 0) {
           throw new RuntimeException("dv for field: " + fieldName + " has ords out of order: " + lastValue + " >=" + term);
@@ -2107,54 +2111,30 @@ public final class CheckIndex implements Closeable {
     }
   }
   
-  private static void checkSortedSetDocValues(String fieldName, int maxDoc, SortedSetDocValues dv, Bits docsWithField) {
+  private static void checkSortedSetDocValues(String fieldName, int maxDoc, SortedSetDocValues dv) throws IOException {
     final long maxOrd = dv.getValueCount()-1;
     LongBitSet seenOrds = new LongBitSet(dv.getValueCount());
     long maxOrd2 = -1;
-    for (int i = 0; i < maxDoc; i++) {
-      dv.setDocument(i);
+    int docID;
+    // TODO: check advance too
+    while ((docID = dv.nextDoc()) != NO_MORE_DOCS) {
       long lastOrd = -1;
       long ord;
-      if (docsWithField.get(i)) {
-        int ordCount = 0;
-        while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-          if (ord <= lastOrd) {
-            throw new RuntimeException("ords out of order: " + ord + " <= " + lastOrd + " for doc: " + i);
-          }
-          if (ord < 0 || ord > maxOrd) {
-            throw new RuntimeException("ord out of bounds: " + ord);
-          }
-          if (dv instanceof RandomAccessOrds) {
-            long ord2 = ((RandomAccessOrds)dv).ordAt(ordCount);
-            if (ord != ord2) {
-              throw new RuntimeException("ordAt(" + ordCount + ") inconsistent, expected=" + ord + ",got=" + ord2 + " for doc: " + i);
-            }
-          }
-          lastOrd = ord;
-          maxOrd2 = Math.max(maxOrd2, ord);
-          seenOrds.set(ord);
-          ordCount++;
-        }
-        if (ordCount == 0) {
-          throw new RuntimeException("dv for field: " + fieldName + " has no ordinals but is not marked missing for doc: " + i);
-        }
-        if (dv instanceof RandomAccessOrds) {
-          long ordCount2 = ((RandomAccessOrds)dv).cardinality();
-          if (ordCount != ordCount2) {
-            throw new RuntimeException("cardinality inconsistent, expected=" + ordCount + ",got=" + ordCount2 + " for doc: " + i);
-          }
+      int ordCount = 0;
+      while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        if (ord <= lastOrd) {
+          throw new RuntimeException("ords out of order: " + ord + " <= " + lastOrd + " for doc: " + docID);
         }
-      } else {
-        long o = dv.nextOrd();
-        if (o != SortedSetDocValues.NO_MORE_ORDS) {
-          throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has ord=" + o + " for doc: " + i);
-        }
-        if (dv instanceof RandomAccessOrds) {
-          long ordCount2 = ((RandomAccessOrds)dv).cardinality();
-          if (ordCount2 != 0) {
-            throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has cardinality " + ordCount2 + " for doc: " + i);
-          }
+        if (ord < 0 || ord > maxOrd) {
+          throw new RuntimeException("ord out of bounds: " + ord);
         }
+        lastOrd = ord;
+        maxOrd2 = Math.max(maxOrd2, ord);
+        seenOrds.set(ord);
+        ordCount++;
+      }
+      if (ordCount == 0) {
+        throw new RuntimeException("dv for field: " + fieldName + " returned docID=" + docID + " yet has no ordinals");
       }
     }
     if (maxOrd != maxOrd2) {
@@ -2177,66 +2157,65 @@ public final class CheckIndex implements Closeable {
     }
   }
   
-  private static void checkSortedNumericDocValues(String fieldName, int maxDoc, SortedNumericDocValues ndv, Bits docsWithField) {
-    for (int i = 0; i < maxDoc; i++) {
-      ndv.setDocument(i);
-      int count = ndv.count();
-      if (docsWithField.get(i)) {
-        if (count == 0) {
-          throw new RuntimeException("dv for field: " + fieldName + " is not marked missing but has zero count for doc: " + i);
-        }
-        long previous = Long.MIN_VALUE;
-        for (int j = 0; j < count; j++) {
-          long value = ndv.valueAt(j);
-          if (value < previous) {
-            throw new RuntimeException("values out of order: " + value + " < " + previous + " for doc: " + i);
-          }
-          previous = value;
-        }
-      } else {
-        if (count != 0) {
-          throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has count=" + count + " for doc: " + i);
+  private static void checkSortedNumericDocValues(String fieldName, int maxDoc, SortedNumericDocValues ndv) throws IOException {
+    if (ndv.docID() != -1) {
+      throw new RuntimeException("dv iterator for field: " + fieldName + " should start at docID=-1, but got " + ndv.docID());
+    }
+    // TODO: check advance too
+    while (true) {
+      int docID = ndv.nextDoc();
+      if (docID == NO_MORE_DOCS) {
+        break;
+      }
+      int count = ndv.docValueCount();
+      if (count == 0) {
+        throw new RuntimeException("sorted numeric dv for field: " + fieldName + " returned docValueCount=0 for docID=" + docID);
+      }
+      long previous = Long.MIN_VALUE;
+      for (int j = 0; j < count; j++) {
+        long value = ndv.nextValue();
+        if (value < previous) {
+          throw new RuntimeException("values out of order: " + value + " < " + previous + " for doc: " + docID);
         }
+        previous = value;
       }
     }
   }
 
-  private static void checkNumericDocValues(String fieldName, int maxDoc, NumericDocValues ndv, Bits docsWithField) {
-    for (int i = 0; i < maxDoc; i++) {
-      long value = ndv.get(i);
-      if (docsWithField.get(i) == false && value != 0) {
-        throw new RuntimeException("dv for field: " + fieldName + " is marked missing but has value=" + value + " for doc: " + i);
-      }
+  private static void checkNumericDocValues(String fieldName, NumericDocValues ndv) throws IOException {
+    int doc;
+    if (ndv.docID() != -1) {
+      throw new RuntimeException("dv iterator for field: " + fieldName + " should start at docID=-1, but got " + ndv.docID());
+    }
+    // TODO: we could add stats to DVs, e.g. total doc count w/ a value for this field
+    // TODO: check advance too
+    while ((doc = ndv.nextDoc()) != NO_MORE_DOCS) {
+      ndv.longValue();
     }
   }
   
   private static void checkDocValues(FieldInfo fi, DocValuesProducer dvReader, int maxDoc, PrintStream infoStream, DocValuesStatus status) throws Exception {
-    Bits docsWithField = dvReader.getDocsWithField(fi);
-    if (docsWithField == null) {
-      throw new RuntimeException(fi.name + " docsWithField does not exist");
-    } else if (docsWithField.length() != maxDoc) {
-      throw new RuntimeException(fi.name + " docsWithField has incorrect length: " + docsWithField.length() + ",expected: " + maxDoc);
-    }
     switch(fi.getDocValuesType()) {
       case SORTED:
         status.totalSortedFields++;
-        checkSortedDocValues(fi.name, maxDoc, dvReader.getSorted(fi), docsWithField);
+        checkBinaryDocValues(fi.name, maxDoc, dvReader.getSorted(fi));
+        checkSortedDocValues(fi.name, maxDoc, dvReader.getSorted(fi));
         break;
       case SORTED_NUMERIC:
         status.totalSortedNumericFields++;
-        checkSortedNumericDocValues(fi.name, maxDoc, dvReader.getSortedNumeric(fi), docsWithField);
+        checkSortedNumericDocValues(fi.name, maxDoc, dvReader.getSortedNumeric(fi));
         break;
       case SORTED_SET:
         status.totalSortedSetFields++;
-        checkSortedSetDocValues(fi.name, maxDoc, dvReader.getSortedSet(fi), docsWithField);
+        checkSortedSetDocValues(fi.name, maxDoc, dvReader.getSortedSet(fi));
         break;
       case BINARY:
         status.totalBinaryFields++;
-        checkBinaryDocValues(fi.name, maxDoc, dvReader.getBinary(fi), docsWithField);
+        checkBinaryDocValues(fi.name, maxDoc, dvReader.getBinary(fi));
         break;
       case NUMERIC:
         status.totalNumericFields++;
-        checkNumericDocValues(fi.name, maxDoc, dvReader.getNumeric(fi), docsWithField);
+        checkNumericDocValues(fi.name, dvReader.getNumeric(fi));
         break;
       default:
         throw new AssertionError();
@@ -2464,8 +2443,9 @@ public final class CheckIndex implements Closeable {
    */
   public void exorciseIndex(Status result) throws IOException {
     ensureOpen();
-    if (result.partial)
+    if (result.partial) {
       throw new IllegalArgumentException("can only exorcise an index that was fully checked (this status checked a subset of segments)");
+    }
     result.newSegments.changed();
     result.newSegments.commit(result.dir);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CodecReader.java b/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
index 194acd8e..b9ad73a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
@@ -21,9 +21,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.FieldsProducer;
@@ -33,9 +31,6 @@ import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.CloseableThreadLocal;
-import org.apache.lucene.util.IOUtils;
 
 /**
  * LeafReader implemented by codec APIs.
@@ -109,20 +104,6 @@ public abstract class CodecReader extends LeafReader implements Accountable {
     return getPostingsReader();
   }
   
-  final CloseableThreadLocal<Map<String,Object>> docValuesLocal = new CloseableThreadLocal<Map<String,Object>>() {
-    @Override
-    protected Map<String,Object> initialValue() {
-      return new HashMap<>();
-    }
-  };
-
-  final CloseableThreadLocal<Map<String,Bits>> docsWithFieldLocal = new CloseableThreadLocal<Map<String,Bits>>() {
-    @Override
-    protected Map<String,Bits> initialValue() {
-      return new HashMap<>();
-    }
-  };
-  
   // returns the FieldInfo that corresponds to the given field and type, or
   // null if the field does not exist, or not indexed as the requested
   // DovDocValuesType.
@@ -143,48 +124,15 @@ public abstract class CodecReader extends LeafReader implements Accountable {
 
     return fi;
   }
-  
-  @Override
-  public final NumericDocValues getNumericDocValues(String field) throws IOException {
-    ensureOpen();
-    Map<String,Object> dvFields = docValuesLocal.get();
-
-    Object previous = dvFields.get(field);
-    if (previous != null && previous instanceof NumericDocValues) {
-      return (NumericDocValues) previous;
-    } else {
-      FieldInfo fi = getDVField(field, DocValuesType.NUMERIC);
-      if (fi == null) {
-        return null;
-      }
-      NumericDocValues dv = getDocValuesReader().getNumeric(fi);
-      dvFields.put(field, dv);
-      return dv;
-    }
-  }
 
   @Override
-  public final Bits getDocsWithField(String field) throws IOException {
+  public final NumericDocValues getNumericDocValues(String field) throws IOException {
     ensureOpen();
-    Map<String,Bits> dvFields = docsWithFieldLocal.get();
-
-    Bits previous = dvFields.get(field);
-    if (previous != null) {
-      return previous;
-    } else {
-      FieldInfo fi = getFieldInfos().fieldInfo(field);
-      if (fi == null) {
-        // Field does not exist
-        return null;
-      }
-      if (fi.getDocValuesType() == DocValuesType.NONE) {
-        // Field was not indexed with doc values
-        return null;
-      }
-      Bits dv = getDocValuesReader().getDocsWithField(fi);
-      dvFields.put(field, dv);
-      return dv;
+    FieldInfo fi = getDVField(field, DocValuesType.NUMERIC);
+    if (fi == null) {
+      return null;
     }
+    return getDocValuesReader().getNumeric(fi);
   }
 
   @Override
@@ -194,105 +142,54 @@ public abstract class CodecReader extends LeafReader implements Accountable {
     if (fi == null) {
       return null;
     }
-
-    Map<String,Object> dvFields = docValuesLocal.get();
-
-    BinaryDocValues dvs = (BinaryDocValues) dvFields.get(field);
-    if (dvs == null) {
-      dvs = getDocValuesReader().getBinary(fi);
-      dvFields.put(field, dvs);
-    }
-
-    return dvs;
+    return getDocValuesReader().getBinary(fi);
   }
 
   @Override
   public final SortedDocValues getSortedDocValues(String field) throws IOException {
     ensureOpen();
-    Map<String,Object> dvFields = docValuesLocal.get();
-    
-    Object previous = dvFields.get(field);
-    if (previous != null && previous instanceof SortedDocValues) {
-      return (SortedDocValues) previous;
-    } else {
-      FieldInfo fi = getDVField(field, DocValuesType.SORTED);
-      if (fi == null) {
-        return null;
-      }
-      SortedDocValues dv = getDocValuesReader().getSorted(fi);
-      dvFields.put(field, dv);
-      return dv;
+    FieldInfo fi = getDVField(field, DocValuesType.SORTED);
+    if (fi == null) {
+      return null;
     }
+    return getDocValuesReader().getSorted(fi);
   }
   
   @Override
   public final SortedNumericDocValues getSortedNumericDocValues(String field) throws IOException {
     ensureOpen();
-    Map<String,Object> dvFields = docValuesLocal.get();
 
-    Object previous = dvFields.get(field);
-    if (previous != null && previous instanceof SortedNumericDocValues) {
-      return (SortedNumericDocValues) previous;
-    } else {
-      FieldInfo fi = getDVField(field, DocValuesType.SORTED_NUMERIC);
-      if (fi == null) {
-        return null;
-      }
-      SortedNumericDocValues dv = getDocValuesReader().getSortedNumeric(fi);
-      dvFields.put(field, dv);
-      return dv;
+    FieldInfo fi = getDVField(field, DocValuesType.SORTED_NUMERIC);
+    if (fi == null) {
+      return null;
     }
+    return getDocValuesReader().getSortedNumeric(fi);
   }
 
   @Override
   public final SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
     ensureOpen();
-    Map<String,Object> dvFields = docValuesLocal.get();
-    
-    Object previous = dvFields.get(field);
-    if (previous != null && previous instanceof SortedSetDocValues) {
-      return (SortedSetDocValues) previous;
-    } else {
-      FieldInfo fi = getDVField(field, DocValuesType.SORTED_SET);
-      if (fi == null) {
-        return null;
-      }
-      SortedSetDocValues dv = getDocValuesReader().getSortedSet(fi);
-      dvFields.put(field, dv);
-      return dv;
+    FieldInfo fi = getDVField(field, DocValuesType.SORTED_SET);
+    if (fi == null) {
+      return null;
     }
+    return getDocValuesReader().getSortedSet(fi);
   }
   
-  final CloseableThreadLocal<Map<String,NumericDocValues>> normsLocal = new CloseableThreadLocal<Map<String,NumericDocValues>>() {
-    @Override
-    protected Map<String,NumericDocValues> initialValue() {
-      return new HashMap<>();
-    }
-  };
-  
   @Override
   public final NumericDocValues getNormValues(String field) throws IOException {
     ensureOpen();
-    Map<String,NumericDocValues> normFields = normsLocal.get();
-
-    NumericDocValues norms = normFields.get(field);
-    if (norms != null) {
-      return norms;
-    } else {
-      FieldInfo fi = getFieldInfos().fieldInfo(field);
-      if (fi == null || !fi.hasNorms()) {
-        // Field does not exist or does not index norms
-        return null;
-      }
-      norms = getNormsReader().getNorms(fi);
-      normFields.put(field, norms);
-      return norms;
+    FieldInfo fi = getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.hasNorms() == false) {
+      // Field does not exist or does not index norms
+      return null;
     }
+
+    return getNormsReader().getNorms(fi);
   }
 
   @Override
   protected void doClose() throws IOException {
-    IOUtils.close(docValuesLocal, docsWithFieldLocal, normsLocal);
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
index 0fb0e3b..cd4726b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocIDMerger.java
@@ -17,6 +17,7 @@
 
 package org.apache.lucene.index;
 
+import java.io.IOException;
 import java.util.List;
 
 import org.apache.lucene.search.DocIdSetIterator; // javadocs
@@ -53,11 +54,11 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
     }
 
     /** Returns the next document ID from this sub reader, and {@link DocIdSetIterator#NO_MORE_DOCS} when done */
-    public abstract int nextDoc();
+    public abstract int nextDoc() throws IOException;
   }
 
   /** Construct this from the provided subs, specifying the maximum sub count */
-  public DocIDMerger(List<T> subs, int maxCount, boolean indexIsSorted) {
+  public DocIDMerger(List<T> subs, int maxCount, boolean indexIsSorted) throws IOException {
     this.subs = subs;
 
     if (indexIsSorted && maxCount > 1) {
@@ -77,12 +78,12 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
   }
 
   /** Construct this from the provided subs */
-  public DocIDMerger(List<T> subs, boolean indexIsSorted) {
+  public DocIDMerger(List<T> subs, boolean indexIsSorted) throws IOException {
     this(subs, subs.size(), indexIsSorted);
   }
 
   /** Reuse API, currently only used by postings during merge */
-  public void reset() {
+  public void reset() throws IOException {
     if (queue != null) {
       // caller may not have fully consumed the queue:
       queue.clear();
@@ -116,7 +117,7 @@ public class DocIDMerger<T extends DocIDMerger.Sub> {
   }
 
   /** Returns null when done */
-  public T next() {
+  public T next() throws IOException {
     // Loop until we find a non-deleted document
     if (queue != null) {
       T top = queue.top();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/DocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValues.java b/lucene/core/src/java/org/apache/lucene/index/DocValues.java
index 4de4238..3377f45 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocValues.java
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
 /** 
@@ -32,25 +31,80 @@ public final class DocValues {
   private DocValues() {}
 
   /** 
-   * An empty BinaryDocValues which returns {@link BytesRef#EMPTY_BYTES} for every document 
+   * An empty {@link BinaryDocValues} which returns no documents
    */
   public static final BinaryDocValues emptyBinary() {
-    final BytesRef empty = new BytesRef();
     return new BinaryDocValues() {
+      private boolean exhausted = false;
+      
       @Override
-      public BytesRef get(int docID) {
-        return empty;
+      public int advance(int target) {
+        assert exhausted == false;
+        assert target >= 0;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public int docID() {
+        return exhausted ? NO_MORE_DOCS : -1;
+      }
+      
+      @Override
+      public int nextDoc() {
+        assert exhausted == false;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public long cost() {
+        return 0;
+      }
+
+      @Override
+      public BytesRef binaryValue() {
+        assert false;
+        return null;
       }
     };
   }
 
   /** 
-   * An empty NumericDocValues which returns zero for every document 
+   * An empty NumericDocValues which returns no documents
    */
   public static final NumericDocValues emptyNumeric() {
     return new NumericDocValues() {
+      private boolean exhausted = false;
+      
+      @Override
+      public int advance(int target) {
+        assert exhausted == false;
+        assert target >= 0;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public int docID() {
+        return exhausted ? NO_MORE_DOCS : -1;
+      }
+      
+      @Override
+      public int nextDoc() {
+        assert exhausted == false;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
       @Override
-      public long get(int docID) {
+      public long cost() {
+        return 0;
+      }
+
+      @Override
+      public long longValue() {
+        assert false;
         return 0;
       }
     };
@@ -59,11 +113,63 @@ public final class DocValues {
   /** 
    * An empty SortedDocValues which returns {@link BytesRef#EMPTY_BYTES} for every document 
    */
+  public static final LegacySortedDocValues emptyLegacySorted() {
+    final BytesRef empty = new BytesRef();
+    return new LegacySortedDocValues() {
+      @Override
+      public int getOrd(int docID) {
+        return -1;
+      }
+
+      @Override
+      public BytesRef lookupOrd(int ord) {
+        return empty;
+      }
+
+      @Override
+      public int getValueCount() {
+        return 0;
+      }
+    };
+  }
+
+  /** 
+   * An empty SortedDocValues which returns {@link BytesRef#EMPTY_BYTES} for every document
+   */
   public static final SortedDocValues emptySorted() {
     final BytesRef empty = new BytesRef();
     return new SortedDocValues() {
+      
+      private boolean exhausted = false;
+      
       @Override
-      public int getOrd(int docID) {
+      public int advance(int target) {
+        assert exhausted == false;
+        assert target >= 0;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public int docID() {
+        return exhausted ? NO_MORE_DOCS : -1;
+      }
+      
+      @Override
+      public int nextDoc() {
+        assert exhausted == false;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public long cost() {
+        return 0;
+      }
+
+      @Override
+      public int ordValue() {
+        assert false;
         return -1;
       }
 
@@ -83,26 +189,109 @@ public final class DocValues {
    * An empty SortedNumericDocValues which returns zero values for every document 
    */
   public static final SortedNumericDocValues emptySortedNumeric(int maxDoc) {
-    return singleton(emptyNumeric(), new Bits.MatchNoBits(maxDoc));
+    return new SortedNumericDocValues() {
+      
+      private boolean exhausted = false;
+      
+      @Override
+      public int advance(int target) {
+        assert exhausted == false;
+        assert target >= 0;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public int docID() {
+        return exhausted ? NO_MORE_DOCS : -1;
+      }
+      
+      @Override
+      public int nextDoc() {
+        assert exhausted == false;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public long cost() {
+        return 0;
+      }
+
+      @Override
+      public int docValueCount() {
+        throw new IllegalStateException();
+      }
+
+      @Override
+      public long nextValue() {
+        throw new IllegalStateException();
+      }
+    };
   }
 
   /** 
-   * An empty SortedDocValues which returns {@link SortedSetDocValues#NO_MORE_ORDS} for every document 
+   * An empty SortedDocValues which returns {@link BytesRef#EMPTY_BYTES} for every document
    */
-  public static final RandomAccessOrds emptySortedSet() {
-    return singleton(emptySorted());
+  public static final SortedSetDocValues emptySortedSet() {
+    final BytesRef empty = new BytesRef();
+    return new SortedSetDocValues() {
+      
+      private boolean exhausted = false;
+      
+      @Override
+      public int advance(int target) {
+        assert exhausted == false;
+        assert target >= 0;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public int docID() {
+        return exhausted ? NO_MORE_DOCS : -1;
+      }
+      
+      @Override
+      public int nextDoc() {
+        assert exhausted == false;
+        exhausted = true;
+        return NO_MORE_DOCS;
+      }
+      
+      @Override
+      public long cost() {
+        return 0;
+      }
+
+      @Override
+      public long nextOrd() {
+        assert false;
+        return NO_MORE_ORDS;
+      }
+
+      @Override
+      public BytesRef lookupOrd(long ord) {
+        return empty;
+      }
+
+      @Override
+      public long getValueCount() {
+        return 0;
+      }
+    };
   }
-  
+
   /** 
-   * Returns a multi-valued view over the provided SortedDocValues 
+   * Returns a multi-valued view over the provided SortedDocValues
    */
-  public static RandomAccessOrds singleton(SortedDocValues dv) {
+  public static SortedSetDocValues singleton(SortedDocValues dv) {
     return new SingletonSortedSetDocValues(dv);
   }
   
   /** 
    * Returns a single-valued view of the SortedSetDocValues, if it was previously
-   * wrapped with {@link #singleton(SortedDocValues)}, or null. 
+   * wrapped with {@link #singleton(SortedDocValues)}, or null.
    */
   public static SortedDocValues unwrapSingleton(SortedSetDocValues dv) {
     if (dv instanceof SingletonSortedSetDocValues) {
@@ -114,8 +303,7 @@ public final class DocValues {
   
   /** 
    * Returns a single-valued view of the SortedNumericDocValues, if it was previously
-   * wrapped with {@link #singleton(NumericDocValues, Bits)}, or null. 
-   * @see #unwrapSingletonBits(SortedNumericDocValues)
+   * wrapped with {@link #singleton(NumericDocValues)}, or null.
    */
   public static NumericDocValues unwrapSingleton(SortedNumericDocValues dv) {
     if (dv instanceof SingletonSortedNumericDocValues) {
@@ -125,76 +313,11 @@ public final class DocValues {
     }
   }
   
-  /** 
-   * Returns the documents with a value for the SortedNumericDocValues, if it was previously
-   * wrapped with {@link #singleton(NumericDocValues, Bits)}, or null. 
-   */
-  public static Bits unwrapSingletonBits(SortedNumericDocValues dv) {
-    if (dv instanceof SingletonSortedNumericDocValues) {
-      return ((SingletonSortedNumericDocValues)dv).getDocsWithField();
-    } else {
-      return null;
-    }
-  }
-  
   /**
    * Returns a multi-valued view over the provided NumericDocValues
    */
-  public static SortedNumericDocValues singleton(NumericDocValues dv, Bits docsWithField) {
-    return new SingletonSortedNumericDocValues(dv, docsWithField);
-  }
-  
-  /**
-   * Returns a Bits representing all documents from <code>dv</code> that have a value.
-   */
-  public static Bits docsWithValue(final SortedDocValues dv, final int maxDoc) {
-    return new Bits() {
-      @Override
-      public boolean get(int index) {
-        return dv.getOrd(index) >= 0;
-      }
-
-      @Override
-      public int length() {
-        return maxDoc;
-      }
-    };
-  }
-  
-  /**
-   * Returns a Bits representing all documents from <code>dv</code> that have a value.
-   */
-  public static Bits docsWithValue(final SortedSetDocValues dv, final int maxDoc) {
-    return new Bits() {
-      @Override
-      public boolean get(int index) {
-        dv.setDocument(index);
-        return dv.nextOrd() != SortedSetDocValues.NO_MORE_ORDS;
-      }
-
-      @Override
-      public int length() {
-        return maxDoc;
-      }
-    };
-  }
-  
-  /**
-   * Returns a Bits representing all documents from <code>dv</code> that have a value.
-   */
-  public static Bits docsWithValue(final SortedNumericDocValues dv, final int maxDoc) {
-    return new Bits() {
-      @Override
-      public boolean get(int index) {
-        dv.setDocument(index);
-        return dv.count() != 0;
-      }
-
-      @Override
-      public int length() {
-        return maxDoc;
-      }
-    };
+  public static SortedNumericDocValues singleton(NumericDocValues dv) {
+    return new SingletonSortedNumericDocValues(dv);
   }
   
   // some helpers, for transition from fieldcache apis.
@@ -215,7 +338,7 @@ public final class DocValues {
   }
   
   /**
-   * Returns NumericDocValues for the field, or {@link #emptyNumeric()} if it has none. 
+   * Returns NumericDocValues for the field, or {@link #emptyNumeric()} if it has none.
    * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
    * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
    * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#NUMERIC}.
@@ -232,7 +355,7 @@ public final class DocValues {
   }
   
   /**
-   * Returns BinaryDocValues for the field, or {@link #emptyBinary} if it has none. 
+   * Returns BinaryDocValues for the field, or {@link #emptyBinary} if it has none.
    * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
    * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
    * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#BINARY}
@@ -252,7 +375,7 @@ public final class DocValues {
   }
   
   /**
-   * Returns SortedDocValues for the field, or {@link #emptySorted} if it has none. 
+   * Returns SortedDocValues for the field, or {@link #emptySorted} if it has none.
    * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
    * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
    * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#SORTED}.
@@ -269,7 +392,7 @@ public final class DocValues {
   }
   
   /**
-   * Returns SortedNumericDocValues for the field, or {@link #emptySortedNumeric} if it has none. 
+   * Returns SortedNumericDocValues for the field, or {@link #emptySortedNumeric} if it has none.
    * @return docvalues instance, or an empty instance if {@code field} does not exist in this reader.
    * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
    * @throws IllegalStateException if {@code field} has docvalues, but the type is not {@link DocValuesType#SORTED_NUMERIC}
@@ -284,8 +407,7 @@ public final class DocValues {
         checkField(reader, field, DocValuesType.SORTED_NUMERIC, DocValuesType.NUMERIC);
         return emptySortedNumeric(reader.maxDoc());
       }
-      Bits bits = reader.getDocsWithField(field);
-      return singleton(single, bits);
+      return singleton(single);
     }
     return dv;
   }
@@ -306,29 +428,8 @@ public final class DocValues {
         checkField(reader, field, DocValuesType.SORTED, DocValuesType.SORTED_SET);
         return emptySortedSet();
       }
-      return singleton(sorted);
+      dv = singleton(sorted);
     }
     return dv;
   }
-  
-  /**
-   * Returns Bits for the field, or {@link Bits} matching nothing if it has none. 
-   * @return bits instance, or an empty instance if {@code field} does not exist in this reader.
-   * @throws IllegalStateException if {@code field} exists, but was not indexed with docvalues.
-   * @throws IOException if an I/O error occurs.
-   */
-  public static Bits getDocsWithField(LeafReader reader, String field) throws IOException {
-    Bits dv = reader.getDocsWithField(field);
-    if (dv == null) {
-      assert DocValuesType.values().length == 6; // we just don't want NONE
-      checkField(reader, field, DocValuesType.BINARY, 
-                            DocValuesType.NUMERIC, 
-                            DocValuesType.SORTED, 
-                            DocValuesType.SORTED_NUMERIC, 
-                            DocValuesType.SORTED_SET);
-      return new Bits.MatchNoBits(reader.maxDoc());
-    } else {
-      return dv;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/EmptyDocValuesProducer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/EmptyDocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/index/EmptyDocValuesProducer.java
new file mode 100644
index 0000000..a4b9049
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/EmptyDocValuesProducer.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesProducer;
+
+/** Abstrast base class implementing a {@link DocValuesProducer} that has no doc values. */
+public abstract class EmptyDocValuesProducer extends DocValuesProducer {
+
+  /** Sole constructor */
+  protected EmptyDocValuesProducer() {
+  }
+  
+  @Override
+  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SortedDocValues getSorted(FieldInfo field) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SortedNumericDocValues getSortedNumeric(FieldInfo field) {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void checkIntegrity() {
+    throw new UnsupportedOperationException();
+  }
+
+  /** Closes this doc values producer. */
+  @Override
+  public void close() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/FilterBinaryDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterBinaryDocValues.java b/lucene/core/src/java/org/apache/lucene/index/FilterBinaryDocValues.java
new file mode 100644
index 0000000..26eb12b
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterBinaryDocValues.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+import java.io.IOException;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Delegates all methods to a wrapped {@link BinaryDocValues}.
+ */
+public abstract class FilterBinaryDocValues extends BinaryDocValues {
+
+  /** Wrapped values */
+  protected final BinaryDocValues in;
+  
+  /** Sole constructor */
+  protected FilterBinaryDocValues(BinaryDocValues in) {
+    this.in = in;
+  }
+
+  @Override
+  public int docID() {
+    return in.docID();
+  }
+  
+  @Override
+  public int nextDoc() throws IOException {
+    return in.nextDoc();
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    return in.advance(target);
+  }
+  
+  @Override
+  public long cost() {
+    return in.cost();
+  }
+
+  @Override
+  public BytesRef binaryValue() {
+    return in.binaryValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
index 886c12a..9e9fcb4 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
@@ -467,12 +467,6 @@ public abstract class FilterLeafReader extends LeafReader {
   }
 
   @Override
-  public Bits getDocsWithField(String field) throws IOException {
-    ensureOpen();
-    return in.getDocsWithField(field);
-  }
-
-  @Override
   public Sort getIndexSort() {
     ensureOpen();
     return in.getIndexSort();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/FilterNumericDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterNumericDocValues.java b/lucene/core/src/java/org/apache/lucene/index/FilterNumericDocValues.java
new file mode 100644
index 0000000..b128d22
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterNumericDocValues.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.index;
+
+import java.io.IOException;
+
+/**
+ * Delegates all methods to a wrapped {@link NumericDocValues}.
+ */
+public abstract class FilterNumericDocValues extends NumericDocValues {
+
+  /** Wrapped values */
+  protected final NumericDocValues in;
+  
+  /** Sole constructor */
+  protected FilterNumericDocValues(NumericDocValues in) {
+    this.in = in;
+  }
+
+  @Override
+  public int docID() {
+    return in.docID();
+  }
+  
+  @Override
+  public int nextDoc() throws IOException {
+    return in.nextDoc();
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    return in.advance(target);
+  }
+  
+  @Override
+  public long cost() {
+    return in.cost();
+  }
+
+  @Override
+  public long longValue() {
+    return in.longValue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
index 44e61e2..fbabaf1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
@@ -16,7 +16,6 @@
  */
 package org.apache.lucene.index;
 
-
 import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader.ReaderClosedListener;
@@ -241,13 +240,13 @@ public abstract class LeafReader extends IndexReader {
   }
 
   /** Returns {@link NumericDocValues} for this field, or
-   *  null if no {@link NumericDocValues} were indexed for
+   *  null if no numeric doc values were indexed for
    *  this field.  The returned instance should only be
-   *  used by a single thread. */
+   *  used by a single thread.  This will never return null. */
   public abstract NumericDocValues getNumericDocValues(String field) throws IOException;
 
   /** Returns {@link BinaryDocValues} for this field, or
-   *  null if no {@link BinaryDocValues} were indexed for
+   *  null if no binary doc values were indexed for
    *  this field.  The returned instance should only be
    *  used by a single thread. */
   public abstract BinaryDocValues getBinaryDocValues(String field) throws IOException;
@@ -270,12 +269,6 @@ public abstract class LeafReader extends IndexReader {
    *  used by a single thread. */
   public abstract SortedSetDocValues getSortedSetDocValues(String field) throws IOException;
 
-  /** Returns a {@link Bits} at the size of <code>reader.maxDoc()</code>,
-   *  with turned on bits for each docid that does have a value for this field,
-   *  or null if no DocValues were indexed for this field. The
-   *  returned instance should only be used by a single thread */
-  public abstract Bits getDocsWithField(String field) throws IOException;
-
   /** Returns {@link NumericDocValues} representing norms
    *  for this field, or null if no {@link NumericDocValues}
    *  were indexed. The returned instance should only be

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7aa200d/lucene/core/src/java/org/apache/lucene/index/LegacyBinaryDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacyBinaryDocValues.java b/lucene/core/src/java/org/apache/lucene/index/LegacyBinaryDocValues.java
new file mode 100644
index 0000000..7aeff7f
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/LegacyBinaryDocValues.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.index;
+
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A per-document byte[]
+ *
+ * @deprecated Use {@link BinaryDocValues} instead.
+ */
+@Deprecated
+public abstract class LegacyBinaryDocValues {
+  
+  /** Sole constructor. (For invocation by subclass 
+   * constructors, typically implicit.) */
+  protected LegacyBinaryDocValues() {}
+
+  /** Lookup the value for document.  The returned {@link BytesRef} may be
+   * re-used across calls to {@link #get(int)} so make sure to
+   * {@link BytesRef#deepCopyOf(BytesRef) copy it} if you want to keep it
+   * around. */
+  public abstract BytesRef get(int docID);
+}