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/08/20 23:30:58 UTC

svn commit: r1515977 [2/4] - in /lucene/dev/trunk: ./ dev-tools/ lucene/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/ lucene/codecs/src/java/org/apache/lucene/codecs/memory/ lucene/codecs/src/java/org/apache/lucene/codecs/simp...

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java Tue Aug 20 21:30:55 2013
@@ -41,7 +41,7 @@ import org.apache.lucene.util.packed.Pac
  * </ul>
  * @see Lucene42DocValuesFormat
  */
-public final class Lucene42NormsFormat extends NormsFormat {
+public class Lucene42NormsFormat extends NormsFormat {
   final float acceptableOverheadRatio;
 
   /** 
@@ -67,7 +67,7 @@ public final class Lucene42NormsFormat e
   
   @Override
   public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
-    return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+    return new Lucene42NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
   }
   
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html Tue Aug 20 21:30:55 2013
@@ -178,7 +178,7 @@ For each field in each document, a value
 that is multiplied into the score for hits on that field.
 </li>
 <li>
-{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vectors}. 
+{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vectors}. 
 For each field in each document, the term vector (sometimes
 called document vector) may be stored. A term vector consists of term text and
 term frequency. To add Term Vectors to your index see the 
@@ -299,17 +299,17 @@ systems that frequently run out of file 
 <td>Encodes additional scoring factors or other per-document information.</td>
 </tr>
 <tr>
-<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Index}</td>
+<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Index}</td>
 <td>.tvx</td>
 <td>Stores offset into the document data file</td>
 </tr>
 <tr>
-<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Documents}</td>
+<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Documents}</td>
 <td>.tvd</td>
 <td>Contains information about each document that has term vectors</td>
 </tr>
 <tr>
-<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Fields}</td>
+<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Fields}</td>
 <td>.tvf</td>
 <td>The field level info about term vectors</td>
 </tr>

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/package.html?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/package.html Tue Aug 20 21:30:55 2013
@@ -61,9 +61,13 @@ name of your codec.
   If you just want to customise the {@link org.apache.lucene.codecs.PostingsFormat}, or use different postings
   formats for different fields, then you can register your custom postings format in the same way (in
   META-INF/services/org.apache.lucene.codecs.PostingsFormat), and then extend the default
-  {@link org.apache.lucene.codecs.lucene42.Lucene42Codec} and override
-  {@link org.apache.lucene.codecs.lucene42.Lucene42Codec#getPostingsFormatForField(String)} to return your custom
+  {@link org.apache.lucene.codecs.lucene45.Lucene45Codec} and override
+  {@link org.apache.lucene.codecs.lucene45.Lucene45Codec#getPostingsFormatForField(String)} to return your custom
   postings format.
 </p>
+<p>
+  Similarly, if you just want to customise the {@link org.apache.lucene.codecs.DocValuesFormat} per-field, have 
+  a look at {@link org.apache.lucene.codecs.lucene45.Lucene45Codec#getDocValuesFormatForField(String)}.
+</p>
 </body>
 </html>

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java Tue Aug 20 21:30:55 2013
@@ -36,6 +36,7 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
@@ -265,6 +266,12 @@ public abstract class PerFieldDocValuesF
       DocValuesProducer producer = fields.get(field.name);
       return producer == null ? null : producer.getSortedSet(field);
     }
+    
+    @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 {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Tue Aug 20 21:30:55 2013
@@ -207,6 +207,12 @@ public abstract class AtomicReader exten
    *  this field.  The returned instance should only be
    *  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}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java Tue Aug 20 21:30:55 2013
@@ -26,6 +26,8 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
 import org.apache.lucene.util.packed.PackedInts;
@@ -49,6 +51,7 @@ class BinaryDocValuesWriter extends DocV
 
   private final Counter iwBytesUsed;
   private final AppendingDeltaPackedLongBuffer lengths;
+  private final OpenBitSet docsWithField;
   private final FieldInfo fieldInfo;
   private int addedValues;
   private long bytesUsed;
@@ -59,6 +62,9 @@ class BinaryDocValuesWriter extends DocV
     this.bytesOut = bytes.getDataOutput();
     this.lengths = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
     this.iwBytesUsed = iwBytesUsed;
+    this.docsWithField = new OpenBitSet();
+    this.bytesUsed = docsWithFieldBytesUsed();
+    iwBytesUsed.addAndGet(bytesUsed);
   }
 
   public void addValue(int docID, BytesRef value) {
@@ -85,11 +91,17 @@ class BinaryDocValuesWriter extends DocV
       // Should never happen!
       throw new RuntimeException(ioe);
     }
+    docsWithField.set(docID);
     updateBytesUsed();
   }
+  
+  private long docsWithFieldBytesUsed() {
+    // size of the long[] + some overhead
+    return RamUsageEstimator.sizeOf(docsWithField.getBits()) + 64;
+  }
 
   private void updateBytesUsed() {
-    final long newBytesUsed = lengths.ramBytesUsed() + bytes.ramBytesUsed();
+    final long newBytesUsed = lengths.ramBytesUsed() + bytes.ramBytesUsed() + docsWithFieldBytesUsed();
     iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
     bytesUsed = newBytesUsed;
   }
@@ -138,6 +150,7 @@ class BinaryDocValuesWriter extends DocV
       if (!hasNext()) {
         throw new NoSuchElementException();
       }
+      final BytesRef v;
       if (upto < size) {
         int length = (int) lengthsIterator.next();
         value.grow(length);
@@ -148,13 +161,16 @@ class BinaryDocValuesWriter extends DocV
           // Should never happen!
           throw new RuntimeException(ioe);
         }
+        if (docsWithField.get(upto)) {
+          v = value;
+        } else {
+          v = null;
+        }
       } else {
-        // This is to handle last N documents not having
-        // this DV field in the end of the segment:
-        value.length = 0;
+        v = null;
       }
       upto++;
-      return value;
+      return v;
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Tue Aug 20 21:30:55 2013
@@ -1280,7 +1280,8 @@ public class CheckIndex {
           if (reader.getBinaryDocValues(fieldInfo.name) != null ||
               reader.getNumericDocValues(fieldInfo.name) != null ||
               reader.getSortedDocValues(fieldInfo.name) != null || 
-              reader.getSortedSetDocValues(fieldInfo.name) != null) {
+              reader.getSortedSetDocValues(fieldInfo.name) != null || 
+              reader.getDocsWithField(fieldInfo.name) != null) {
             throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
           }
         }
@@ -1301,26 +1302,37 @@ public class CheckIndex {
     return status;
   }
   
-  private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv) {
+  private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv, Bits docsWithField) {
     BytesRef scratch = new BytesRef();
     for (int i = 0; i < reader.maxDoc(); i++) {
       dv.get(i, scratch);
       assert scratch.isValid();
+      if (docsWithField.get(i) == false && scratch.length > 0) {
+        throw new RuntimeException("dv for field: " + fieldName + " is missing but has value=" + scratch + " for doc: " + i);
+      }
     }
   }
   
-  private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv) {
-    checkBinaryDocValues(fieldName, reader, dv);
+  private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv, Bits docsWithField) {
+    checkBinaryDocValues(fieldName, reader, dv, docsWithField);
     final int maxOrd = dv.getValueCount()-1;
     FixedBitSet seenOrds = new FixedBitSet(dv.getValueCount());
     int maxOrd2 = -1;
     for (int i = 0; i < reader.maxDoc(); i++) {
       int ord = dv.getOrd(i);
-      if (ord < 0 || ord > maxOrd) {
+      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);
+        }
+      } 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);
       }
-      maxOrd2 = Math.max(maxOrd2, ord);
-      seenOrds.set(ord);
     }
     if (maxOrd != maxOrd2) {
       throw new RuntimeException("dv for field: " + fieldName + " reports wrong maxOrd=" + maxOrd + " but this is not the case: " + maxOrd2);
@@ -1342,7 +1354,7 @@ public class CheckIndex {
     }
   }
   
-  private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv) {
+  private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv, Bits docsWithField) {
     final long maxOrd = dv.getValueCount()-1;
     OpenBitSet seenOrds = new OpenBitSet(dv.getValueCount());
     long maxOrd2 = -1;
@@ -1350,16 +1362,28 @@ public class CheckIndex {
       dv.setDocument(i);
       long lastOrd = -1;
       long ord;
-      while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
-        if (ord <= lastOrd) {
-          throw new RuntimeException("ords out of order: " + ord + " <= " + lastOrd + " for doc: " + i);
+      if (docsWithField.get(i)) {
+        int ordCount = 0;
+        while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+          ordCount++;
+          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);
+          }
+          lastOrd = ord;
+          maxOrd2 = Math.max(maxOrd2, ord);
+          seenOrds.set(ord);
         }
-        if (ord < 0 || ord > maxOrd) {
-          throw new RuntimeException("ord out of bounds: " + ord);
+        if (ordCount == 0) {
+          throw new RuntimeException("dv for field: " + fieldName + " has no ordinals but is not marked missing for doc: " + i);
+        }
+      } 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);
         }
-        lastOrd = ord;
-        maxOrd2 = Math.max(maxOrd2, ord);
-        seenOrds.set(ord);
       }
     }
     if (maxOrd != maxOrd2) {
@@ -1383,17 +1407,26 @@ public class CheckIndex {
     }
   }
 
-  private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv) {
+  private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv, Bits docsWithField) {
     for (int i = 0; i < reader.maxDoc(); i++) {
-      ndv.get(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 checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream, DocValuesStatus status) throws Exception {
+    Bits docsWithField = reader.getDocsWithField(fi.name);
+    if (docsWithField == null) {
+      throw new RuntimeException(fi.name + " docsWithField does not exist");
+    } else if (docsWithField.length() != reader.maxDoc()) {
+      throw new RuntimeException(fi.name + " docsWithField has incorrect length: " + docsWithField.length() + ",expected: " + reader.maxDoc());
+    }
     switch(fi.getDocValuesType()) {
       case SORTED:
         status.totalSortedFields++;
-        checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
+        checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name), docsWithField);
         if (reader.getBinaryDocValues(fi.name) != null ||
             reader.getNumericDocValues(fi.name) != null ||
             reader.getSortedSetDocValues(fi.name) != null) {
@@ -1402,7 +1435,7 @@ public class CheckIndex {
         break;
       case SORTED_SET:
         status.totalSortedSetFields++;
-        checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name));
+        checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name), docsWithField);
         if (reader.getBinaryDocValues(fi.name) != null ||
             reader.getNumericDocValues(fi.name) != null ||
             reader.getSortedDocValues(fi.name) != null) {
@@ -1411,7 +1444,7 @@ public class CheckIndex {
         break;
       case BINARY:
         status.totalBinaryFields++;
-        checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
+        checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name), docsWithField);
         if (reader.getNumericDocValues(fi.name) != null ||
             reader.getSortedDocValues(fi.name) != null ||
             reader.getSortedSetDocValues(fi.name) != null) {
@@ -1420,7 +1453,7 @@ public class CheckIndex {
         break;
       case NUMERIC:
         status.totalNumericFields++;
-        checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
+        checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name), docsWithField);
         if (reader.getBinaryDocValues(fi.name) != null ||
             reader.getSortedDocValues(fi.name) != null ||
             reader.getSortedSetDocValues(fi.name) != null) {
@@ -1435,7 +1468,7 @@ public class CheckIndex {
   private static void checkNorms(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws IOException {
     switch(fi.getNormType()) {
       case NUMERIC:
-        checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name));
+        checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name), new Bits.MatchAllBits(reader.maxDoc()));
         break;
       default:
         throw new AssertionError("wtf: " + fi.getNormType());

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java Tue Aug 20 21:30:55 2013
@@ -143,7 +143,7 @@ final class DocValuesProcessor extends S
     DocValuesWriter writer = writers.get(fieldInfo.name);
     NumericDocValuesWriter numericWriter;
     if (writer == null) {
-      numericWriter = new NumericDocValuesWriter(fieldInfo, bytesUsed);
+      numericWriter = new NumericDocValuesWriter(fieldInfo, bytesUsed, true);
       writers.put(fieldInfo.name, numericWriter);
     } else if (!(writer instanceof NumericDocValuesWriter)) {
       throw new IllegalArgumentException("Incompatible DocValues type: field \"" + fieldInfo.name + "\" changed from " + getTypeDesc(writer) + " to numeric");

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Tue Aug 20 21:30:55 2013
@@ -414,4 +414,10 @@ public class FilterAtomicReader extends 
     return in.getNormValues(field);
   }
 
+  @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    ensureOpen();
+    return in.getDocsWithField(field);
+  }
+
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Tue Aug 20 21:30:55 2013
@@ -22,6 +22,7 @@ import java.util.List;
 
 import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
 import org.apache.lucene.index.MultiTermsEnum.TermsEnumWithSlice;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
 import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
@@ -135,6 +136,51 @@ public class MultiDocValues {
       };
     }
   }
+  
+  /** Returns a Bits for a reader's docsWithField (potentially merging on-the-fly) 
+   * <p>
+   * This is a slow way to access this bitset. Instead, access them per-segment
+   * with {@link AtomicReader#getDocsWithField(String)}
+   * </p> 
+   * */
+  public static Bits getDocsWithField(final IndexReader r, final String field) throws IOException {
+    final List<AtomicReaderContext> leaves = r.leaves();
+    final int size = leaves.size();
+    if (size == 0) {
+      return null;
+    } else if (size == 1) {
+      return leaves.get(0).reader().getDocsWithField(field);
+    }
+
+    boolean anyReal = false;
+    boolean anyMissing = false;
+    final Bits[] values = new Bits[size];
+    final int[] starts = new int[size+1];
+    for (int i = 0; i < size; i++) {
+      AtomicReaderContext context = leaves.get(i);
+      Bits v = context.reader().getDocsWithField(field);
+      if (v == null) {
+        v = new Bits.MatchNoBits(context.reader().maxDoc());
+        anyMissing = true;
+      } else {
+        anyReal = true;
+        if (v instanceof Bits.MatchAllBits == false) {
+          anyMissing = true;
+        }
+      }
+      values[i] = v;
+      starts[i] = context.docBase;
+    }
+    starts[size] = r.maxDoc();
+
+    if (!anyReal) {
+      return null;
+    } else if (!anyMissing) {
+      return new Bits.MatchAllBits(r.maxDoc());
+    } else {
+      return new MultiBits(values, starts, false);
+    }
+  }
 
   /** Returns a BinaryDocValues for a reader's docvalues (potentially merging on-the-fly)
    * <p>

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java Tue Aug 20 21:30:55 2013
@@ -44,7 +44,7 @@ final class NormsConsumerPerField extend
     if (fieldInfo.isIndexed() && !fieldInfo.omitsNorms()) {
       if (consumer == null) {
         fieldInfo.setNormValueType(FieldInfo.DocValuesType.NUMERIC);
-        consumer = new NumericDocValuesWriter(fieldInfo, docState.docWriter.bytesUsed);
+        consumer = new NumericDocValuesWriter(fieldInfo, docState.docWriter.bytesUsed, false);
       }
       consumer.addValue(docState.docID, similarity.computeNorm(fieldState));
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java Tue Aug 20 21:30:55 2013
@@ -23,6 +23,8 @@ import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -35,14 +37,18 @@ class NumericDocValuesWriter extends Doc
   private AppendingDeltaPackedLongBuffer pending;
   private final Counter iwBytesUsed;
   private long bytesUsed;
+  private final OpenBitSet docsWithField;
   private final FieldInfo fieldInfo;
+  private final boolean trackDocsWithField;
 
-  public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
+  public NumericDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed, boolean trackDocsWithField) {
     pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
-    bytesUsed = pending.ramBytesUsed();
+    docsWithField = new OpenBitSet();
+    bytesUsed = pending.ramBytesUsed() + docsWithFieldBytesUsed();
     this.fieldInfo = fieldInfo;
     this.iwBytesUsed = iwBytesUsed;
     iwBytesUsed.addAndGet(bytesUsed);
+    this.trackDocsWithField = trackDocsWithField;
   }
 
   public void addValue(int docID, long value) {
@@ -56,12 +62,20 @@ class NumericDocValuesWriter extends Doc
     }
 
     pending.add(value);
+    if (trackDocsWithField) {
+      docsWithField.set(docID);
+    }
 
     updateBytesUsed();
   }
+  
+  private long docsWithFieldBytesUsed() {
+    // size of the long[] + some overhead
+    return RamUsageEstimator.sizeOf(docsWithField.getBits()) + 64;
+  }
 
   private void updateBytesUsed() {
-    final long newBytesUsed = pending.ramBytesUsed();
+    final long newBytesUsed = pending.ramBytesUsed() + docsWithFieldBytesUsed();
     iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
     bytesUsed = newBytesUsed;
   }
@@ -109,14 +123,18 @@ class NumericDocValuesWriter extends Doc
       if (!hasNext()) {
         throw new NoSuchElementException();
       }
-      long value;
+      Long value;
       if (upto < size) {
-        value = iter.next();
+        long v = iter.next();
+        if (!trackDocsWithField || docsWithField.get(upto)) {
+          value = v;
+        } else {
+          value = null;
+        }
       } else {
-        value = 0;
+        value = trackDocsWithField ? null : MISSING;
       }
       upto++;
-      // TODO: make reusable Number
       return value;
     }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java Tue Aug 20 21:30:55 2013
@@ -286,6 +286,13 @@ public class ParallelAtomicReader extend
   }
 
   @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    ensureOpen();
+    AtomicReader reader = fieldToReader.get(field);
+    return reader == null ? null : reader.getDocsWithField(field);
+  }
+
+  @Override
   public NumericDocValues getNormValues(String field) throws IOException {
     ensureOpen();
     AtomicReader reader = fieldToReader.get(field);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Tue Aug 20 21:30:55 2013
@@ -36,6 +36,7 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.CloseableThreadLocal;
 import org.apache.lucene.util.IOUtils;
 
@@ -87,6 +88,13 @@ final class SegmentCoreReaders {
       return new HashMap<String,Object>();
     }
   };
+  
+  final CloseableThreadLocal<Map<String,Bits>> docsWithFieldLocal = new CloseableThreadLocal<Map<String,Bits>>() {
+    @Override
+    protected Map<String,Bits> initialValue() {
+      return new HashMap<String,Bits>();
+    }
+  };
 
   final CloseableThreadLocal<Map<String,Object>> normsLocal = new CloseableThreadLocal<Map<String,Object>>() {
     @Override
@@ -274,6 +282,30 @@ final class SegmentCoreReaders {
 
     return dvs;
   }
+  
+  Bits getDocsWithField(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;
+    }
+
+    assert dvProducer != null;
+
+    Map<String,Bits> dvFields = docsWithFieldLocal.get();
+
+    Bits dvs = dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getDocsWithField(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
+  }
 
   NumericDocValues getNormValues(String field) throws IOException {
     FieldInfo fi = fieldInfos.fieldInfo(field);
@@ -300,8 +332,8 @@ final class SegmentCoreReaders {
 
   void decRef() throws IOException {
     if (ref.decrementAndGet() == 0) {
-      IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, fields, dvProducer,
-                    termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
+      IOUtils.close(termVectorsLocal, fieldsReaderLocal, docValuesLocal, normsLocal, docsWithFieldLocal, fields, 
+                    dvProducer, termVectorsReaderOrig, fieldsReaderOrig, cfsReader, normsProducer);
       notifyCoreClosedListeners();
     }
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Tue Aug 20 21:30:55 2013
@@ -30,6 +30,7 @@ import org.apache.lucene.codecs.TermVect
 import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
 
@@ -156,24 +157,32 @@ final class SegmentMerger {
         if (type != null) {
           if (type == DocValuesType.NUMERIC) {
             List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
+            List<Bits> docsWithField = new ArrayList<Bits>();
             for (AtomicReader reader : mergeState.readers) {
               NumericDocValues values = reader.getNumericDocValues(field.name);
+              Bits bits = reader.getDocsWithField(field.name);
               if (values == null) {
                 values = NumericDocValues.EMPTY;
+                bits = new Bits.MatchNoBits(reader.maxDoc());
               }
               toMerge.add(values);
+              docsWithField.add(bits);
             }
-            consumer.mergeNumericField(field, mergeState, toMerge);
+            consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
           } else if (type == DocValuesType.BINARY) {
             List<BinaryDocValues> toMerge = new ArrayList<BinaryDocValues>();
+            List<Bits> docsWithField = new ArrayList<Bits>();
             for (AtomicReader reader : mergeState.readers) {
               BinaryDocValues values = reader.getBinaryDocValues(field.name);
+              Bits bits = reader.getDocsWithField(field.name);
               if (values == null) {
                 values = BinaryDocValues.EMPTY;
+                bits = new Bits.MatchNoBits(reader.maxDoc());
               }
               toMerge.add(values);
+              docsWithField.add(bits);
             }
-            consumer.mergeBinaryField(field, mergeState, toMerge);
+            consumer.mergeBinaryField(field, mergeState, toMerge, docsWithField);
           } else if (type == DocValuesType.SORTED) {
             List<SortedDocValues> toMerge = new ArrayList<SortedDocValues>();
             for (AtomicReader reader : mergeState.readers) {
@@ -216,14 +225,16 @@ final class SegmentMerger {
       for (FieldInfo field : mergeState.fieldInfos) {
         if (field.hasNorms()) {
           List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
+          List<Bits> docsWithField = new ArrayList<Bits>();
           for (AtomicReader reader : mergeState.readers) {
             NumericDocValues norms = reader.getNormValues(field.name);
             if (norms == null) {
               norms = NumericDocValues.EMPTY;
             }
             toMerge.add(norms);
+            docsWithField.add(new Bits.MatchAllBits(reader.maxDoc()));
           }
-          consumer.mergeNumericField(field, mergeState, toMerge);
+          consumer.mergeNumericField(field, mergeState, toMerge, docsWithField);
         }
       }
       success = true;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Tue Aug 20 21:30:55 2013
@@ -224,6 +224,12 @@ public final class SegmentReader extends
   }
 
   @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    ensureOpen();
+    return core.getDocsWithField(field);
+  }
+
+  @Override
   public BinaryDocValues getBinaryDocValues(String field) throws IOException {
     ensureOpen();
     return core.getBinaryDocValues(field);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Tue Aug 20 21:30:55 2013
@@ -92,6 +92,12 @@ public final class SlowCompositeReaderWr
   }
 
   @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    ensureOpen();
+    return MultiDocValues.getDocsWithField(in, field);
+  }
+
+  @Override
   public BinaryDocValues getBinaryDocValues(String field) throws IOException {
     ensureOpen();
     return MultiDocValues.getBinaryValues(in, field);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java Tue Aug 20 21:30:55 2013
@@ -37,12 +37,13 @@ public abstract class SortedDocValues ex
    * Returns the ordinal for the specified docID.
    * @param  docID document ID to lookup
    * @return ordinal for the document: this is dense, starts at 0, then
-   *         increments by 1 for the next value in sorted order. 
+   *         increments by 1 for the next value in sorted order. Note that
+   *         missing values are indicated by -1.
    */
   public abstract int getOrd(int docID);
 
   /** Retrieves the value for the specified ordinal.
-   * @param ord ordinal to lookup
+   * @param ord ordinal to lookup (must be &gt;= 0 and &lt {@link #getValueCount()})
    * @param result will be populated with the ordinal's value
    * @see #getOrd(int) 
    */
@@ -71,7 +72,7 @@ public abstract class SortedDocValues ex
   public static final SortedDocValues EMPTY = new SortedDocValues() {
     @Override
     public int getOrd(int docID) {
-      return 0;
+      return -1;
     }
 
     @Override
@@ -83,7 +84,7 @@ public abstract class SortedDocValues ex
 
     @Override
     public int getValueCount() {
-      return 1;
+      return 0;
     }
   };
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java Tue Aug 20 21:30:55 2013
@@ -30,19 +30,19 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.packed.AppendingPackedLongBuffer;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
 import org.apache.lucene.util.packed.PackedInts;
 
 /** Buffers up pending byte[] per doc, deref and sorting via
  *  int ord, then flushes when segment flushes. */
 class SortedDocValuesWriter extends DocValuesWriter {
   final BytesRefHash hash;
-  private AppendingPackedLongBuffer pending;
+  private AppendingDeltaPackedLongBuffer pending;
   private final Counter iwBytesUsed;
   private long bytesUsed; // this currently only tracks differences in 'pending'
   private final FieldInfo fieldInfo;
 
-  private static final BytesRef EMPTY = new BytesRef(BytesRef.EMPTY_BYTES);
+  private static final int EMPTY_ORD = -1;
 
   public SortedDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
     this.fieldInfo = fieldInfo;
@@ -52,7 +52,7 @@ class SortedDocValuesWriter extends DocV
             new ByteBlockPool.DirectTrackingAllocator(iwBytesUsed)),
             BytesRefHash.DEFAULT_CAPACITY,
             new DirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, iwBytesUsed));
-    pending = new AppendingPackedLongBuffer(PackedInts.COMPACT);
+    pending = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
     bytesUsed = pending.ramBytesUsed();
     iwBytesUsed.addAndGet(bytesUsed);
   }
@@ -70,7 +70,7 @@ class SortedDocValuesWriter extends DocV
 
     // Fill in any holes:
     while(pending.size() < docID) {
-      addOneValue(EMPTY);
+      pending.add(EMPTY_ORD);
     }
 
     addOneValue(value);
@@ -79,8 +79,9 @@ class SortedDocValuesWriter extends DocV
   @Override
   public void finish(int maxDoc) {
     while(pending.size() < maxDoc) {
-      addOneValue(EMPTY);
+      pending.add(EMPTY_ORD);
     }
+    updateBytesUsed();
   }
 
   private void addOneValue(BytesRef value) {
@@ -177,7 +178,7 @@ class SortedDocValuesWriter extends DocV
   
   // iterates over the ords for each doc we have in ram
   private class OrdsIterator implements Iterator<Number> {
-    final AppendingPackedLongBuffer.Iterator iter = pending.iterator();
+    final AppendingDeltaPackedLongBuffer.Iterator iter = pending.iterator();
     final int ordMap[];
     final int maxDoc;
     int docUpto;
@@ -200,8 +201,7 @@ class SortedDocValuesWriter extends DocV
       }
       int ord = (int) iter.next();
       docUpto++;
-      // TODO: make reusable Number
-      return ordMap[ord];
+      return ord == -1 ? ord : ordMap[ord];
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCache.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCache.java Tue Aug 20 21:30:55 2013
@@ -104,26 +104,6 @@ public interface FieldCache {
       }
     };
   }
-  
-  /** Returns MISSING/-1 ordinal for every document */
-  public static final SortedDocValues EMPTY_TERMSINDEX = new SortedDocValues() {
-    @Override
-    public int getOrd(int docID) {
-      return -1;
-    }
-
-    @Override
-    public void lookupOrd(int ord, BytesRef result) {
-      result.bytes = MISSING;
-      result.offset = 0;
-      result.length = 0;
-    }
-
-    @Override
-    public int getValueCount() {
-      return 0;
-    }
-  };
 
   /**
    * Placeholder indicating creation of this cache is currently in-progress.
@@ -266,13 +246,10 @@ 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.  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).
+   *  does have a value for this field.
    */
   public Bits getDocsWithField(AtomicReader reader, String field) throws IOException;
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Tue Aug 20 21:30:55 2013
@@ -501,8 +501,7 @@ class FieldCacheImpl implements FieldCac
       // field does not exist or has no value
       return new Bits.MatchNoBits(reader.maxDoc());
     } else if (fieldInfo.hasDocValues()) {
-      // doc values are dense
-      return new Bits.MatchAllBits(reader.maxDoc());
+      return reader.getDocsWithField(field);
     } else if (!fieldInfo.isIndexed()) {
       return new Bits.MatchNoBits(reader.maxDoc());
     }
@@ -944,13 +943,13 @@ class FieldCacheImpl implements FieldCac
     } else {
       final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
       if (info == null) {
-        return EMPTY_TERMSINDEX;
+        return SortedDocValues.EMPTY;
       } else if (info.hasDocValues()) {
         // we don't try to build a sorted instance from numeric/binary doc
         // values because dedup can be very costly
         throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
       } else if (!info.isIndexed()) {
-        return EMPTY_TERMSINDEX;
+        return SortedDocValues.EMPTY;
       }
       return (SortedDocValues) caches.get(SortedDocValues.class).get(reader, new CacheKey(field, acceptableOverheadRatio), false);
     }

Modified: lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Tue Aug 20 21:30:55 2013
@@ -16,3 +16,4 @@
 org.apache.lucene.codecs.lucene40.Lucene40Codec
 org.apache.lucene.codecs.lucene41.Lucene41Codec
 org.apache.lucene.codecs.lucene42.Lucene42Codec
+org.apache.lucene.codecs.lucene45.Lucene45Codec
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/trunk/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Tue Aug 20 21:30:55 2013
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat
+org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java Tue Aug 20 21:30:55 2013
@@ -17,21 +17,27 @@ package org.apache.lucene;
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.*;
-import org.apache.lucene.codecs.*;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
-import org.apache.lucene.document.*;
-import org.apache.lucene.index.*;
-import org.apache.lucene.search.*;
-import org.apache.lucene.store.*;
-import org.apache.lucene.util.*;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.BaseDirectoryWrapper;
+import org.apache.lucene.util.LuceneTestCase;
+
 
 /* Intentionally outside of oal.index to verify fully
    external codecs work fine */
 
 public class TestExternalCodecs extends LuceneTestCase {
 
-  private static final class CustomPerFieldCodec extends Lucene42Codec {
+  private static final class CustomPerFieldCodec extends Lucene45Codec {
     
     private final PostingsFormat ramFormat = PostingsFormat.forName("RAMOnly");
     private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene42/TestLucene42DocValuesFormat.java Tue Aug 20 21:30:55 2013
@@ -24,7 +24,7 @@ import org.apache.lucene.index.BaseCompr
  * Tests Lucene42DocValuesFormat
  */
 public class TestLucene42DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
-  private final Codec codec = new Lucene42Codec();
+  private final Codec codec = new Lucene42RWCodec();
 
   @Override
   protected Codec getCodec() {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java Tue Aug 20 21:30:55 2013
@@ -25,7 +25,7 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -79,9 +79,9 @@ public class TestPerFieldDocValuesFormat
     Directory directory = newDirectory();
     // we don't use RandomIndexWriter because it might add more docvalues than we expect !!!!1
     IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
-    final DocValuesFormat fast = DocValuesFormat.forName("Lucene42");
+    final DocValuesFormat fast = DocValuesFormat.forName("Lucene45");
     final DocValuesFormat slow = DocValuesFormat.forName("SimpleText");
-    iwc.setCodec(new Lucene42Codec() {
+    iwc.setCodec(new Lucene45Codec() {
       @Override
       public DocValuesFormat getDocValuesFormatForField(String field) {
         if ("dv1".equals(field)) {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java Tue Aug 20 21:30:55 2013
@@ -21,7 +21,7 @@ import java.io.IOException;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
 import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat;
 import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
@@ -200,7 +200,7 @@ public class TestPerFieldPostingsFormat2
 
   }
 
-  public static class MockCodec extends Lucene42Codec {
+  public static class MockCodec extends Lucene45Codec {
     final PostingsFormat lucene40 = new Lucene41PostingsFormat();
     final PostingsFormat simpleText = new SimpleTextPostingsFormat();
     final PostingsFormat mockSep = new MockSepPostingsFormat();
@@ -217,7 +217,7 @@ public class TestPerFieldPostingsFormat2
     }
   }
 
-  public static class MockCodec2 extends Lucene42Codec {
+  public static class MockCodec2 extends Lucene45Codec {
     final PostingsFormat lucene40 = new Lucene41PostingsFormat();
     final PostingsFormat simpleText = new SimpleTextPostingsFormat();
     
@@ -268,7 +268,7 @@ public class TestPerFieldPostingsFormat2
   }
   
   public void testSameCodecDifferentInstance() throws Exception {
-    Codec codec = new Lucene42Codec() {
+    Codec codec = new Lucene45Codec() {
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {
         if ("id".equals(field)) {
@@ -284,7 +284,7 @@ public class TestPerFieldPostingsFormat2
   }
   
   public void testSameCodecDifferentParams() throws Exception {
-    Codec codec = new Lucene42Codec() {
+    Codec codec = new Lucene45Codec() {
       @Override
       public PostingsFormat getPostingsFormatForField(String field) {
         if ("id".equals(field)) {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java Tue Aug 20 21:30:55 2013
@@ -28,7 +28,7 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
 import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -1060,7 +1060,7 @@ public class TestAddIndexes extends Luce
     aux2.close();
   }
 
-  private static final class CustomPerFieldCodec extends Lucene42Codec {
+  private static final class CustomPerFieldCodec extends Lucene45Codec {
     private final PostingsFormat simpleTextFormat = PostingsFormat.forName("SimpleText");
     private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
     private final PostingsFormat mockSepFormat = PostingsFormat.forName("MockSep");
@@ -1111,7 +1111,7 @@ public class TestAddIndexes extends Luce
   
   private static final class UnRegisteredCodec extends FilterCodec {
     public UnRegisteredCodec() {
-      super("NotRegistered", new Lucene42Codec());
+      super("NotRegistered", new Lucene45Codec());
     }
   }
   

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java Tue Aug 20 21:30:55 2013
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+import org.apache.lucene.codecs.lucene45.Lucene45Codec;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexFileNames;
@@ -41,7 +41,7 @@ public class TestAllFilesHaveCodecHeader
   public void test() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    conf.setCodec(new Lucene42Codec());
+    conf.setCodec(new Lucene45Codec());
     // riw should sometimes create docvalues fields, etc
     RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
     Document doc = new Document();

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Tue Aug 20 21:30:55 2013
@@ -74,7 +74,7 @@ import org.junit.Ignore;
 // we won't even be running the actual code, only the impostor
 // @SuppressCodecs("Lucene4x")
 // Sep codec cannot yet handle the offsets in our 4.x index!
-@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41"})
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41", "Lucene42"})
 public class TestBackwardsCompatibility extends LuceneTestCase {
 
   // Uncomment these cases & run them on an older Lucene version,

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java Tue Aug 20 21:30:55 2013
@@ -24,6 +24,7 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
@@ -49,7 +50,7 @@ public class TestDuelingCodecs extends L
   public void setUp() throws Exception {
     super.setUp();
 
-    // for now its SimpleText vs Lucene42(random postings format)
+    // for now its SimpleText vs Lucene45(random postings format)
     // as this gives the best overall coverage. when we have more
     // codecs we should probably pick 2 from Codec.availableCodecs()
     
@@ -134,6 +135,11 @@ public class TestDuelingCodecs extends L
       for (String trash : split) {
         document.add(new SortedSetDocValuesField("sortedset", new BytesRef(trash)));
       }
+      // add a numeric dv field sometimes
+      document.removeFields("sparsenumeric");
+      if (random.nextInt(4) == 2) {
+        document.add(new NumericDocValuesField("sparsenumeric", random.nextInt()));
+      }
       writer.addDocument(document);
     }
     

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java Tue Aug 20 21:30:55 2013
@@ -26,6 +26,7 @@ import org.apache.lucene.document.Numeri
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
@@ -321,4 +322,52 @@ public class TestMultiDocValues extends 
     ir2.close();
     dir.close();
   }
+  
+  public void testDocsWithField() throws Exception {
+    assumeTrue("codec does not support docsWithField", defaultCodecSupportsDocsWithField());
+    Directory dir = newDirectory();
+    
+    IndexWriterConfig iwc = newIndexWriterConfig(random(), TEST_VERSION_CURRENT, null);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+
+    int numDocs = atLeast(500);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      if (random().nextInt(4) >= 0) {
+        doc.add(new NumericDocValuesField("numbers", random().nextLong()));
+      }
+      doc.add(new NumericDocValuesField("numbersAlways", random().nextLong()));
+      iw.addDocument(doc);
+      if (random().nextInt(17) == 0) {
+        iw.commit();
+      }
+    }
+    DirectoryReader ir = iw.getReader();
+    iw.forceMerge(1);
+    DirectoryReader ir2 = iw.getReader();
+    AtomicReader merged = getOnlySegmentReader(ir2);
+    iw.close();
+    
+    Bits multi = MultiDocValues.getDocsWithField(ir, "numbers");
+    Bits single = merged.getDocsWithField("numbers");
+    if (multi == null) {
+      assertNull(single);
+    } else {
+      assertEquals(single.length(), multi.length());
+      for (int i = 0; i < numDocs; i++) {
+        assertEquals(single.get(i), multi.get(i));
+      }
+    }
+    
+    multi = MultiDocValues.getDocsWithField(ir, "numbersAlways");
+    single = merged.getDocsWithField("numbersAlways");
+    assertEquals(single.length(), multi.length());
+    for (int i = 0; i < numDocs; i++) {
+      assertEquals(single.get(i), multi.get(i));
+    }
+    ir.close();
+    ir2.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java Tue Aug 20 21:30:55 2013
@@ -480,7 +480,7 @@ public class TestFieldCache extends Luce
     } catch (IllegalStateException expected) {}
     
     Bits bits = FieldCache.DEFAULT.getDocsWithField(ar, "binary");
-    assertTrue(bits instanceof Bits.MatchAllBits);
+    assertTrue(bits.get(0));
     
     // Sorted type: can be retrieved via getTerms(), getTermsIndex(), getDocTermOrds()
     try {
@@ -510,7 +510,7 @@ public class TestFieldCache extends Luce
     assertEquals(1, sortedSet.getValueCount());
     
     bits = FieldCache.DEFAULT.getDocsWithField(ar, "sorted");
-    assertTrue(bits instanceof Bits.MatchAllBits);
+    assertTrue(bits.get(0));
     
     // Numeric type: can be retrieved via getInts() and so on
     Ints numeric = FieldCache.DEFAULT.getInts(ar, "numeric", false);
@@ -537,7 +537,7 @@ public class TestFieldCache extends Luce
     } catch (IllegalStateException expected) {}
     
     bits = FieldCache.DEFAULT.getDocsWithField(ar, "numeric");
-    assertTrue(bits instanceof Bits.MatchAllBits);
+    assertTrue(bits.get(0));
     
     // SortedSet type: can be retrieved via getDocTermOrds() 
     if (defaultCodecSupportsSortedSet()) {
@@ -569,7 +569,7 @@ public class TestFieldCache extends Luce
       assertEquals(2, sortedSet.getValueCount());
     
       bits = FieldCache.DEFAULT.getDocsWithField(ar, "sortedset");
-      assertTrue(bits instanceof Bits.MatchAllBits);
+      assertTrue(bits.get(0));
     }
     
     ir.close();

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortDocValues.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortDocValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSortDocValues.java Tue Aug 20 21:30:55 2013
@@ -31,10 +31,12 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 
 /** Tests basic sorting on docvalues fields.
  * These are mostly like TestSort's tests, except each test
  * indexes the field up-front as docvalues, and checks no fieldcaches were made */
+@SuppressCodecs({"Lucene40", "Lucene41", "Lucene42"}) // avoid codecs that don't support "missing"
 public class TestSortDocValues extends LuceneTestCase {
   
   @Override
@@ -291,6 +293,70 @@ public class TestSortDocValues extends L
     dir.close();
   }
   
+  /** Tests sorting on type int with a missing value */
+  public void testIntMissing() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new NumericDocValuesField("value", -1));
+    doc.add(newStringField("value", "-1", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new NumericDocValuesField("value", 4));
+    doc.add(newStringField("value", "4", Field.Store.YES));
+    writer.addDocument(doc);
+    IndexReader ir = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    Sort sort = new Sort(new SortField("value", SortField.Type.INT));
+
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
+    assertEquals(3, td.totalHits);
+    // null is treated as a 0
+    assertEquals("-1", searcher.doc(td.scoreDocs[0].doc).get("value"));
+    assertNull(searcher.doc(td.scoreDocs[1].doc).get("value"));
+    assertEquals("4", searcher.doc(td.scoreDocs[2].doc).get("value"));
+
+    ir.close();
+    dir.close();
+  }
+  
+  /** Tests sorting on type int, specifying the missing value should be treated as Integer.MAX_VALUE */
+  public void testIntMissingLast() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new NumericDocValuesField("value", -1));
+    doc.add(newStringField("value", "-1", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new NumericDocValuesField("value", 4));
+    doc.add(newStringField("value", "4", Field.Store.YES));
+    writer.addDocument(doc);
+    IndexReader ir = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    SortField sortField = new SortField("value", SortField.Type.INT);
+    sortField.setMissingValue(Integer.MAX_VALUE);
+    Sort sort = new Sort(sortField);
+
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
+    assertEquals(3, td.totalHits);
+    // null is treated as a Integer.MAX_VALUE
+    assertEquals("-1", searcher.doc(td.scoreDocs[0].doc).get("value"));
+    assertEquals("4", searcher.doc(td.scoreDocs[1].doc).get("value"));
+    assertNull(searcher.doc(td.scoreDocs[2].doc).get("value"));
+
+    ir.close();
+    dir.close();
+  }
+  
   /** Tests sorting on type long */
   public void testLong() throws IOException {
     Directory dir = newDirectory();
@@ -359,6 +425,70 @@ public class TestSortDocValues extends L
     dir.close();
   }
   
+  /** Tests sorting on type long with a missing value */
+  public void testLongMissing() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new NumericDocValuesField("value", -1));
+    doc.add(newStringField("value", "-1", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new NumericDocValuesField("value", 4));
+    doc.add(newStringField("value", "4", Field.Store.YES));
+    writer.addDocument(doc);
+    IndexReader ir = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    Sort sort = new Sort(new SortField("value", SortField.Type.LONG));
+
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
+    assertEquals(3, td.totalHits);
+    // null is treated as 0
+    assertEquals("-1", searcher.doc(td.scoreDocs[0].doc).get("value"));
+    assertNull(searcher.doc(td.scoreDocs[1].doc).get("value"));
+    assertEquals("4", searcher.doc(td.scoreDocs[2].doc).get("value"));
+
+    ir.close();
+    dir.close();
+  }
+  
+  /** Tests sorting on type long, specifying the missing value should be treated as Long.MAX_VALUE */
+  public void testLongMissingLast() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new NumericDocValuesField("value", -1));
+    doc.add(newStringField("value", "-1", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new NumericDocValuesField("value", 4));
+    doc.add(newStringField("value", "4", Field.Store.YES));
+    writer.addDocument(doc);
+    IndexReader ir = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    SortField sortField = new SortField("value", SortField.Type.LONG);
+    sortField.setMissingValue(Long.MAX_VALUE);
+    Sort sort = new Sort(sortField);
+
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
+    assertEquals(3, td.totalHits);
+    // null is treated as Long.MAX_VALUE
+    assertEquals("-1", searcher.doc(td.scoreDocs[0].doc).get("value"));
+    assertEquals("4", searcher.doc(td.scoreDocs[1].doc).get("value"));
+    assertNull(searcher.doc(td.scoreDocs[2].doc).get("value"));
+
+    ir.close();
+    dir.close();
+  }
+  
   /** Tests sorting on type float */
   public void testFloat() throws IOException {
     Directory dir = newDirectory();
@@ -427,6 +557,70 @@ public class TestSortDocValues extends L
     dir.close();
   }
   
+  /** Tests sorting on type float with a missing value */
+  public void testFloatMissing() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new FloatDocValuesField("value", -1.3F));
+    doc.add(newStringField("value", "-1.3", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new FloatDocValuesField("value", 4.2F));
+    doc.add(newStringField("value", "4.2", Field.Store.YES));
+    writer.addDocument(doc);
+    IndexReader ir = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    Sort sort = new Sort(new SortField("value", SortField.Type.FLOAT));
+
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
+    assertEquals(3, td.totalHits);
+    // null is treated as 0
+    assertEquals("-1.3", searcher.doc(td.scoreDocs[0].doc).get("value"));
+    assertNull(searcher.doc(td.scoreDocs[1].doc).get("value"));
+    assertEquals("4.2", searcher.doc(td.scoreDocs[2].doc).get("value"));
+
+    ir.close();
+    dir.close();
+  }
+  
+  /** Tests sorting on type float, specifying the missing value should be treated as Float.MAX_VALUE */
+  public void testFloatMissingLast() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new FloatDocValuesField("value", -1.3F));
+    doc.add(newStringField("value", "-1.3", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new FloatDocValuesField("value", 4.2F));
+    doc.add(newStringField("value", "4.2", Field.Store.YES));
+    writer.addDocument(doc);
+    IndexReader ir = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    SortField sortField = new SortField("value", SortField.Type.FLOAT);
+    sortField.setMissingValue(Float.MAX_VALUE);
+    Sort sort = new Sort(sortField);
+
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
+    assertEquals(3, td.totalHits);
+    // null is treated as Float.MAX_VALUE
+    assertEquals("-1.3", searcher.doc(td.scoreDocs[0].doc).get("value"));
+    assertEquals("4.2", searcher.doc(td.scoreDocs[1].doc).get("value"));
+    assertNull(searcher.doc(td.scoreDocs[2].doc).get("value"));
+
+    ir.close();
+    dir.close();
+  }
+  
   /** Tests sorting on type double */
   public void testDouble() throws IOException {
     Directory dir = newDirectory();
@@ -533,4 +727,78 @@ public class TestSortDocValues extends L
     ir.close();
     dir.close();
   }
+  
+  /** Tests sorting on type double with a missing value */
+  public void testDoubleMissing() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new DoubleDocValuesField("value", -1.3));
+    doc.add(newStringField("value", "-1.3", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new DoubleDocValuesField("value", 4.2333333333333));
+    doc.add(newStringField("value", "4.2333333333333", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new DoubleDocValuesField("value", 4.2333333333332));
+    doc.add(newStringField("value", "4.2333333333332", Field.Store.YES));
+    writer.addDocument(doc);
+    IndexReader ir = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    Sort sort = new Sort(new SortField("value", SortField.Type.DOUBLE));
+
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
+    assertEquals(4, td.totalHits);
+    // null treated as a 0
+    assertEquals("-1.3", searcher.doc(td.scoreDocs[0].doc).get("value"));
+    assertNull(searcher.doc(td.scoreDocs[1].doc).get("value"));
+    assertEquals("4.2333333333332", searcher.doc(td.scoreDocs[2].doc).get("value"));
+    assertEquals("4.2333333333333", searcher.doc(td.scoreDocs[3].doc).get("value"));
+
+    ir.close();
+    dir.close();
+  }
+  
+  /** Tests sorting on type double, specifying the missing value should be treated as Double.MAX_VALUE */
+  public void testDoubleMissingLast() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new DoubleDocValuesField("value", -1.3));
+    doc.add(newStringField("value", "-1.3", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new DoubleDocValuesField("value", 4.2333333333333));
+    doc.add(newStringField("value", "4.2333333333333", Field.Store.YES));
+    writer.addDocument(doc);
+    doc = new Document();
+    doc.add(new DoubleDocValuesField("value", 4.2333333333332));
+    doc.add(newStringField("value", "4.2333333333332", Field.Store.YES));
+    writer.addDocument(doc);
+    IndexReader ir = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    SortField sortField = new SortField("value", SortField.Type.DOUBLE);
+    sortField.setMissingValue(Double.MAX_VALUE);
+    Sort sort = new Sort(sortField);
+
+    TopDocs td = searcher.search(new MatchAllDocsQuery(), 10, sort);
+    assertEquals(4, td.totalHits);
+    // null treated as Double.MAX_VALUE
+    assertEquals("-1.3", searcher.doc(td.scoreDocs[0].doc).get("value"));
+    assertEquals("4.2333333333332", searcher.doc(td.scoreDocs[1].doc).get("value"));
+    assertEquals("4.2333333333333", searcher.doc(td.scoreDocs[2].doc).get("value"));
+    assertNull(searcher.doc(td.scoreDocs[3].doc).get("value"));
+
+    ir.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java Tue Aug 20 21:30:55 2013
@@ -25,8 +25,8 @@ import org.apache.lucene.codecs.Codec;
 // enough to test the basics via Codec
 public class TestNamedSPILoader extends LuceneTestCase {
   public void testLookup() {
-    Codec codec = Codec.forName("Lucene42");
-    assertEquals("Lucene42", codec.getName());
+    Codec codec = Codec.forName("Lucene45");
+    assertEquals("Lucene45", codec.getName());
   }
   
   // we want an exception if its not found.
@@ -39,6 +39,6 @@ public class TestNamedSPILoader extends 
   
   public void testAvailableServices() {
     Set<String> codecs = Codec.availableCodecs();
-    assertTrue(codecs.contains("Lucene42"));
+    assertTrue(codecs.contains("Lucene45"));
   }
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesConsumer.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesConsumer.java Tue Aug 20 21:30:55 2013
@@ -68,7 +68,9 @@ public class Facet42DocValuesConsumer ex
 
     long totBytes = 0;
     for (BytesRef v : values) {
-      totBytes += v.length;
+      if (v != null) { 
+        totBytes += v.length;
+      }
     }
 
     if (totBytes > Integer.MAX_VALUE) {
@@ -78,7 +80,9 @@ public class Facet42DocValuesConsumer ex
     out.writeVInt((int) totBytes);
 
     for (BytesRef v : values) {
-      out.writeBytes(v.bytes, v.offset, v.length);
+      if (v != null) {
+        out.writeBytes(v.bytes, v.offset, v.length);
+      }
     }
 
     PackedInts.Writer w = PackedInts.getWriter(out, maxDoc+1, PackedInts.bitsRequired(totBytes+1), acceptableOverheadRatio);
@@ -86,7 +90,9 @@ public class Facet42DocValuesConsumer ex
     int address = 0;
     for(BytesRef v : values) {
       w.add(address);
-      address += v.length;
+      if (v != null) {
+        address += v.length;
+      }
     }
     w.add(address);
     w.finish();

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesProducer.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesProducer.java Tue Aug 20 21:30:55 2013
@@ -31,15 +31,18 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.IOUtils;
 
 class Facet42DocValuesProducer extends DocValuesProducer {
 
   private final Map<Integer,Facet42BinaryDocValues> fields = new HashMap<Integer,Facet42BinaryDocValues>();
+  private final int maxDoc;
   
   Facet42DocValuesProducer(SegmentReadState state) throws IOException {
     String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Facet42DocValuesFormat.EXTENSION);
     IndexInput in = state.directory.openInput(fileName, state.context);
+    this.maxDoc = state.segmentInfo.getDocCount();
     boolean success = false;
     try {
       CodecUtil.checkHeader(in, Facet42DocValuesFormat.CODEC, 
@@ -81,6 +84,11 @@ class Facet42DocValuesProducer extends D
   }
 
   @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    return new Bits.MatchAllBits(maxDoc); // TODO: have codec impl this?
+  }
+
+  @Override
   public void close() throws IOException {
   }
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java Tue Aug 20 21:30:55 2013
@@ -40,6 +40,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -223,6 +224,18 @@ public class FacetsPayloadMigrationReade
   }
 
   @Override
+  public Bits getDocsWithField(String field) throws IOException {
+    Term term = fieldTerms.get(field);
+    if (term == null) {
+      return super.getDocsWithField(field);
+    } else {
+      // we shouldn't return null, even if the term does not exist or has no
+      // payloads, since we already marked the field as having DocValues.
+      return new Bits.MatchAllBits(maxDoc());
+    }
+  }
+
+  @Override
   public FieldInfos getFieldInfos() {
     FieldInfos innerInfos = super.getFieldInfos();
     ArrayList<FieldInfo> infos = new ArrayList<FieldInfo>(innerInfos.size());

Modified: lucene/dev/trunk/lucene/facet/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/trunk/lucene/facet/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Tue Aug 20 21:30:55 2013
@@ -13,4 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.facet.codecs.facet42.Facet42Codec
+org.apache.lucene.facet.codecs.facet45.Facet45Codec

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestCase.java Tue Aug 20 21:30:55 2013
@@ -3,7 +3,7 @@ package org.apache.lucene.facet;
 import java.util.Random;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.facet.codecs.facet42.Facet42Codec;
+import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
 import org.apache.lucene.facet.encoding.DGapIntEncoder;
 import org.apache.lucene.facet.encoding.DGapVInt8IntEncoder;
 import org.apache.lucene.facet.encoding.EightFlagsIntEncoder;
@@ -53,7 +53,7 @@ public abstract class FacetTestCase exte
   public static void beforeClassFacetTestCase() throws Exception {
     if (random().nextDouble() < 0.3) {
       savedDefault = Codec.getDefault(); // save to restore later
-      Codec.setDefault(new Facet42Codec());
+      Codec.setDefault(new Facet45Codec());
     }
   }
   

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestDemoFacets.java Tue Aug 20 21:30:55 2013
@@ -31,7 +31,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.FacetTestUtils;
-import org.apache.lucene.facet.codecs.facet42.Facet42Codec;
+import org.apache.lucene.facet.codecs.facet45.Facet45Codec;
 import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.params.CategoryListParams;
 import org.apache.lucene.facet.params.FacetIndexingParams;
@@ -260,7 +260,7 @@ public class TestDemoFacets extends Face
     Directory dir = newDirectory();
     Directory taxoDir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
-    iwc.setCodec(new Facet42Codec());
+    iwc.setCodec(new Facet45Codec());
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
     DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
 

Modified: lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java?rev=1515977&r1=1515976&r2=1515977&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java Tue Aug 20 21:30:55 2013
@@ -422,6 +422,11 @@ public class WeightedSpanTermExtractor {
     public NumericDocValues getNormValues(String field) throws IOException {
       return super.getNormValues(FIELD_NAME);
     }
+
+    @Override
+    public Bits getDocsWithField(String field) throws IOException {
+      return super.getDocsWithField(FIELD_NAME);
+    }
   }
 
   /**