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/16 12:25:39 UTC

svn commit: r1514642 [1/2] - in /lucene/dev/branches/lucene5178: lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core/src/java/org...

Author: rmuir
Date: Fri Aug 16 10:25:38 2013
New Revision: 1514642

URL: http://svn.apache.org/r1514642
Log:
LUCENE-5178: add 'missing' support to docvalues (simpletext only)

Added:
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/MissingOrdRemapper.java   (with props)
Modified:
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesWriter.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java
    lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesProducer.java
    lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java
    lucene/dev/branches/lucene5178/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
    lucene/dev/branches/lucene5178/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/branches/lucene5178/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingAtomicReader.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java
    lucene/dev/branches/lucene5178/solr/core/src/java/org/apache/solr/request/DocValuesFacets.java
    lucene/dev/branches/lucene5178/solr/core/src/test/org/apache/solr/search/TestDocSet.java

Modified: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java Fri Aug 16 10:25:38 2013
@@ -23,6 +23,7 @@ import java.util.HashSet;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.MissingOrdRemapper;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
@@ -92,8 +93,9 @@ public class DiskDocValuesConsumer exten
     if (optimizeStorage) {
       uniqueValues = new HashSet<>();
 
+      // nocommit: impl null values (ideally smartly)
       for (Number nv : values) {
-        final long v = nv.longValue();
+        final long v = nv == null ? 0 : nv.longValue();
 
         if (gcd != 1) {
           if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
@@ -151,14 +153,15 @@ public class DiskDocValuesConsumer exten
         meta.writeLong(gcd);
         final BlockPackedWriter quotientWriter = new BlockPackedWriter(data, BLOCK_SIZE);
         for (Number nv : values) {
-          quotientWriter.add((nv.longValue() - minValue) / gcd);
+          long value = nv == null ? 0 : nv.longValue();
+          quotientWriter.add((value - minValue) / gcd);
         }
         quotientWriter.finish();
         break;
       case DELTA_COMPRESSED:
         final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
         for (Number nv : values) {
-          writer.add(nv.longValue());
+          writer.add(nv == null ? 0 : nv.longValue());
         }
         writer.finish();
         break;
@@ -173,7 +176,7 @@ public class DiskDocValuesConsumer exten
         final int bitsRequired = PackedInts.bitsRequired(uniqueValues.size() - 1);
         final PackedInts.Writer ordsWriter = PackedInts.getWriterNoHeader(data, PackedInts.Format.PACKED, (int) count, bitsRequired, PackedInts.DEFAULT_BUFFER_SIZE);
         for (Number nv : values) {
-          ordsWriter.add(encode.get(nv.longValue()));
+          ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
         }
         ordsWriter.finish();
         break;
@@ -192,9 +195,12 @@ public class DiskDocValuesConsumer exten
     final long startFP = data.getFilePointer();
     long count = 0;
     for(BytesRef v : values) {
-      minLength = Math.min(minLength, v.length);
-      maxLength = Math.max(maxLength, v.length);
-      data.writeBytes(v.bytes, v.offset, v.length);
+      final int length = v == null ? 0 : v.length;
+      minLength = Math.min(minLength, length);
+      maxLength = Math.max(maxLength, length);
+      if (v != null) {
+        data.writeBytes(v.bytes, v.offset, v.length);
+      }
       count++;
     }
     meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
@@ -213,7 +219,9 @@ public class DiskDocValuesConsumer exten
       final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
       long addr = 0;
       for (BytesRef v : values) {
-        addr += v.length;
+        if (v != null) {
+          addr += v.length;
+        }
         writer.add(addr);
       }
       writer.finish();
@@ -278,6 +286,34 @@ public class DiskDocValuesConsumer exten
 
   @Override
   public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    // nocommit: remove this hack and support missing!
+
+    // three cases for simulating the old writer:
+    // 1. no missing
+    // 2. missing (and empty string in use): remap ord=-1 -> ord=0
+    // 3. missing (and empty string not in use): remap all ords +1, insert empty string into values
+    boolean anyMissing = false;
+    for (Number n : docToOrd) {
+      if (n.longValue() == -1) {
+        anyMissing = true;
+        break;
+      }
+    }
+    
+    boolean hasEmptyString = false;
+    for (BytesRef b : values) {
+      hasEmptyString = b.length == 0;
+      break;
+    }
+    
+    if (!anyMissing) {
+      // nothing to do
+    } else if (hasEmptyString) {
+      docToOrd = MissingOrdRemapper.mapMissingToOrd0(docToOrd);
+    } else {
+      docToOrd = MissingOrdRemapper.mapAllOrds(docToOrd);
+      values = MissingOrdRemapper.insertEmptyValue(values);
+    }
     meta.writeVInt(field.number);
     meta.writeByte(DiskDocValuesFormat.SORTED);
     addTermsDict(field, values);

Modified: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java Fri Aug 16 10:25:38 2013
@@ -32,6 +32,7 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.DocValuesProducer.SortedSetDocsWithField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsAndPositionsEnum;
@@ -59,6 +60,7 @@ class DiskDocValuesProducer extends DocV
   private final Map<Integer,NumericEntry> ords;
   private final Map<Integer,NumericEntry> ordIndexes;
   private final IndexInput data;
+  private final int maxDoc;
 
   // memory-resident structures
   private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
@@ -68,6 +70,7 @@ class DiskDocValuesProducer extends DocV
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
     // read in the entries from the metadata file.
     IndexInput in = state.directory.openInput(metaName, state.context);
+    this.maxDoc = state.segmentInfo.getDocCount();
     boolean success = false;
     final int version;
     try {
@@ -491,6 +494,17 @@ class DiskDocValuesProducer extends DocV
   }
 
   @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    // nocommit: only use this if the field's entry has missing values (write that),
+    // otherwise return MatchAllBits
+    if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
+      return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
+    } else {
+      return new Bits.MatchAllBits(maxDoc);
+    }
+  }
+
+  @Override
   public void close() throws IOException {
     data.close();
   }

Modified: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java Fri Aug 16 10:25:38 2013
@@ -38,12 +38,16 @@ import org.apache.lucene.index.SegmentWr
  *    minvalue 0
  *    pattern 000
  *  005
+ *  T
  *  234
+ *  T
  *  123
+ *  T
  *  ...
  *  </pre>
  *  so a document's value (delta encoded from minvalue) can be retrieved by 
- *  seeking to startOffset + (1+pattern.length())*docid. The extra 1 is the newline.
+ *  seeking to startOffset + (1+pattern.length()+2)*docid. The extra 1 is the newline. 
+ *  The extra 2 is another newline and 'T' or 'F': true if the value is real, false if missing.
  *  
  *  for bytes this is also a "fixed-width" file, for example:
  *  <pre>
@@ -53,12 +57,15 @@ import org.apache.lucene.index.SegmentWr
  *    pattern 0
  *  length 6
  *  foobar[space][space]
+ *  T
  *  length 3
  *  baz[space][space][space][space][space]
+ *  T
  *  ...
  *  </pre>
- *  so a doc's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength)*doc
+ *  so a doc's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength+2)*doc
  *  the extra 9 is 2 newlines, plus "length " itself.
+ *  the extra 2 is another newline and 'T' or 'F': true if the value is real, false if missing.
  *  
  *  for sorted bytes this is a fixed-width file, for example:
  *  <pre>

Modified: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Fri Aug 16 10:25:38 2013
@@ -28,6 +28,7 @@ import java.util.Locale;
 import java.util.Map;
 
 import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.DocValuesProducer.SortedSetDocsWithField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
@@ -38,6 +39,7 @@ import org.apache.lucene.index.SortedDoc
 import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.StringHelper;
 
@@ -100,7 +102,7 @@ class SimpleTextDocValuesReader extends 
         assert startsWith(PATTERN);
         field.pattern = stripPrefix(PATTERN);
         field.dataStartFilePointer = data.getFilePointer();
-        data.seek(data.getFilePointer() + (1+field.pattern.length()) * maxDoc);
+        data.seek(data.getFilePointer() + (1+field.pattern.length()+2) * maxDoc);
       } else if (dvType == DocValuesType.BINARY) {
         readLine();
         assert startsWith(MAXLENGTH);
@@ -109,7 +111,7 @@ class SimpleTextDocValuesReader extends 
         assert startsWith(PATTERN);
         field.pattern = stripPrefix(PATTERN);
         field.dataStartFilePointer = data.getFilePointer();
-        data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength) * maxDoc);
+        data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength+2) * maxDoc);
       } else if (dvType == DocValuesType.SORTED || dvType == DocValuesType.SORTED_SET) {
         readLine();
         assert startsWith(NUMVALUES);
@@ -158,7 +160,7 @@ class SimpleTextDocValuesReader extends 
           if (docID < 0 || docID >= maxDoc) {
             throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
           }
-          in.seek(field.dataStartFilePointer + (1+field.pattern.length())*docID);
+          in.seek(field.dataStartFilePointer + (1+field.pattern.length()+2)*docID);
           SimpleTextUtil.readLine(in, scratch);
           //System.out.println("parsing delta: " + scratch.utf8ToString());
           BigDecimal bd;
@@ -169,6 +171,7 @@ class SimpleTextDocValuesReader extends 
             e.initCause(pe);
             throw e;
           }
+          SimpleTextUtil.readLine(in, scratch); // read the line telling us if its real or not
           return BigInteger.valueOf(field.minValue).add(bd.toBigIntegerExact()).longValue();
         } catch (IOException ioe) {
           throw new RuntimeException(ioe);
@@ -176,6 +179,30 @@ class SimpleTextDocValuesReader extends 
       }
     };
   }
+  
+  private Bits getNumericDocsWithField(FieldInfo fieldInfo) throws IOException {
+    final OneField field = fields.get(fieldInfo.name);
+    final IndexInput in = data.clone();
+    final BytesRef scratch = new BytesRef();
+    return new Bits() {
+      @Override
+      public boolean get(int index) {
+        try {
+          in.seek(field.dataStartFilePointer + (1+field.pattern.length()+2)*index);
+          SimpleTextUtil.readLine(in, scratch); // data
+          SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
+          return scratch.bytes[scratch.offset] == (byte) 'T';
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      @Override
+      public int length() {
+        return maxDoc;
+      }
+    };
+  }
 
   @Override
   public BinaryDocValues getBinary(FieldInfo fieldInfo) throws IOException {
@@ -196,7 +223,7 @@ class SimpleTextDocValuesReader extends 
           if (docID < 0 || docID >= maxDoc) {
             throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
           }
-          in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength)*docID);
+          in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*docID);
           SimpleTextUtil.readLine(in, scratch);
           assert StringHelper.startsWith(scratch, LENGTH);
           int len;
@@ -217,6 +244,45 @@ class SimpleTextDocValuesReader extends 
       }
     };
   }
+  
+  private Bits getBinaryDocsWithField(FieldInfo fieldInfo) throws IOException {
+    final OneField field = fields.get(fieldInfo.name);
+    final IndexInput in = data.clone();
+    final BytesRef scratch = new BytesRef();
+    final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
+
+    return new Bits() {
+      @Override
+      public boolean get(int index) {
+        try {
+          in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*index);
+          SimpleTextUtil.readLine(in, scratch);
+          assert StringHelper.startsWith(scratch, LENGTH);
+          int len;
+          try {
+            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
+          } catch (ParseException pe) {
+            CorruptIndexException e = new CorruptIndexException("failed to parse int length (resource=" + in + ")");
+            e.initCause(pe);
+            throw e;
+          }
+          // skip past bytes
+          byte bytes[] = new byte[len];
+          in.readBytes(bytes, 0, len);
+          SimpleTextUtil.readLine(in, scratch); // newline
+          SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
+          return scratch.bytes[scratch.offset] == (byte) 'T';
+        } catch (IOException ioe) {
+          throw new RuntimeException(ioe);
+        }
+      }
+
+      @Override
+      public int length() {
+        return maxDoc;
+      }
+    };
+  }
 
   @Override
   public SortedDocValues getSorted(FieldInfo fieldInfo) throws IOException {
@@ -241,7 +307,7 @@ class SimpleTextDocValuesReader extends 
           in.seek(field.dataStartFilePointer + field.numValues * (9 + field.pattern.length() + field.maxLength) + docID * (1 + field.ordPattern.length()));
           SimpleTextUtil.readLine(in, scratch);
           try {
-            return ordDecoder.parse(scratch.utf8ToString()).intValue();
+            return (int) ordDecoder.parse(scratch.utf8ToString()).longValue()-1;
           } catch (ParseException pe) {
             CorruptIndexException e = new CorruptIndexException("failed to parse ord (resource=" + in + ")");
             e.initCause(pe);
@@ -255,8 +321,12 @@ class SimpleTextDocValuesReader extends 
       @Override
       public void lookupOrd(int ord, BytesRef result) {
         try {
-          if (ord < 0 || ord >= field.numValues) {
-            throw new IndexOutOfBoundsException("ord must be 0 .. " + (field.numValues-1) + "; got " + ord);
+          if (ord == -1) {
+            result.length = 0;
+            return;
+          }
+          if (ord < -1 || ord >= field.numValues) {
+            throw new IndexOutOfBoundsException("ord must be -1 .. " + (field.numValues-1) + "; got " + ord);
           }
           in.seek(field.dataStartFilePointer + ord * (9 + field.pattern.length() + field.maxLength));
           SimpleTextUtil.readLine(in, scratch);
@@ -362,6 +432,21 @@ class SimpleTextDocValuesReader extends 
       }
     };
   }
+  
+  @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
+      return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
+    } else if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED) {
+      return new SortedDocsWithField(getSorted(field), maxDoc);
+    } else if (field.getDocValuesType() == FieldInfo.DocValuesType.BINARY) {
+      return getBinaryDocsWithField(field);
+    } else if (field.getDocValuesType() == FieldInfo.DocValuesType.NUMERIC) {
+      return getNumericDocsWithField(field);
+    } else {
+      return new Bits.MatchAllBits(maxDoc);
+    }
+  }
 
   @Override
   public void close() throws IOException {

Modified: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java Fri Aug 16 10:25:38 2013
@@ -78,7 +78,7 @@ class SimpleTextDocValuesWriter extends 
     long minValue = Long.MAX_VALUE;
     long maxValue = Long.MIN_VALUE;
     for(Number n : values) {
-      long v = n.longValue();
+      long v = n == null ? 0 : n.longValue();
       minValue = Math.min(minValue, v);
       maxValue = Math.max(maxValue, v);
     }
@@ -112,13 +112,19 @@ class SimpleTextDocValuesWriter extends 
 
     // second pass to write the values
     for(Number n : values) {
-      long value = n.longValue();
+      long value = n == null ? 0 : n.longValue();
       assert value >= minValue;
       Number delta = BigInteger.valueOf(value).subtract(BigInteger.valueOf(minValue));
       String s = encoder.format(delta);
       assert s.length() == patternString.length();
       SimpleTextUtil.write(data, s, scratch);
       SimpleTextUtil.writeNewline(data);
+      if (n == null) {
+        SimpleTextUtil.write(data, "F", scratch);
+      } else {
+        SimpleTextUtil.write(data, "T", scratch);
+      }
+      SimpleTextUtil.writeNewline(data);
       numDocsWritten++;
       assert numDocsWritten <= numDocs;
     }
@@ -132,7 +138,8 @@ class SimpleTextDocValuesWriter extends 
     assert field.getDocValuesType() == DocValuesType.BINARY;
     int maxLength = 0;
     for(BytesRef value : values) {
-      maxLength = Math.max(maxLength, value.length);
+      final int length = value == null ? 0 : value.length;
+      maxLength = Math.max(maxLength, length);
     }
     writeFieldEntry(field, FieldInfo.DocValuesType.BINARY);
 
@@ -155,19 +162,28 @@ class SimpleTextDocValuesWriter extends 
     int numDocsWritten = 0;
     for(BytesRef value : values) {
       // write length
+      final int length = value == null ? 0 : value.length;
       SimpleTextUtil.write(data, LENGTH);
-      SimpleTextUtil.write(data, encoder.format(value.length), scratch);
+      SimpleTextUtil.write(data, encoder.format(length), scratch);
       SimpleTextUtil.writeNewline(data);
         
       // write bytes -- don't use SimpleText.write
       // because it escapes:
-      data.writeBytes(value.bytes, value.offset, value.length);
+      if (value != null) {
+        data.writeBytes(value.bytes, value.offset, value.length);
+      }
 
       // pad to fit
-      for (int i = value.length; i < maxLength; i++) {
+      for (int i = length; i < maxLength; i++) {
         data.writeByte((byte)' ');
       }
       SimpleTextUtil.writeNewline(data);
+      if (value == null) {
+        SimpleTextUtil.write(data, "F", scratch);
+      } else {
+        SimpleTextUtil.write(data, "T", scratch);
+      }
+      SimpleTextUtil.writeNewline(data);
       numDocsWritten++;
     }
 
@@ -209,7 +225,7 @@ class SimpleTextDocValuesWriter extends 
     SimpleTextUtil.writeNewline(data);
     final DecimalFormat encoder = new DecimalFormat(sb.toString(), new DecimalFormatSymbols(Locale.ROOT));
     
-    int maxOrdBytes = Integer.toString(valueCount).length();
+    int maxOrdBytes = Long.toString(valueCount+1L).length();
     sb.setLength(0);
     for (int i = 0; i < maxOrdBytes; i++) {
       sb.append('0');
@@ -246,7 +262,7 @@ class SimpleTextDocValuesWriter extends 
     assert valuesSeen == valueCount;
 
     for(Number ord : docToOrd) {
-      SimpleTextUtil.write(data, ordEncoder.format(ord.intValue()), scratch);
+      SimpleTextUtil.write(data, ordEncoder.format(ord.longValue()+1), scratch);
       SimpleTextUtil.writeNewline(data);
     }
   }

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Fri Aug 16 10:25:38 2013
@@ -69,7 +69,8 @@ public abstract class DocValuesConsumer 
   /**
    * Writes numeric docvalues for a field.
    * @param field field information
-   * @param values Iterable of numeric values (one for each document).
+   * @param values Iterable of numeric values (one for each document). {@code null} indicates
+   *               a missing value.
    * @throws IOException if an I/O error occurred.
    */
   public abstract void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException;    
@@ -77,7 +78,8 @@ public abstract class DocValuesConsumer 
   /**
    * Writes binary docvalues for a field.
    * @param field field information
-   * @param values Iterable of binary values (one for each document).
+   * @param values Iterable of binary values (one for each document). {@code null} indicates
+   *               a missing value.
    * @throws IOException if an I/O error occurred.
    */
   public abstract void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException;
@@ -86,7 +88,8 @@ public abstract class DocValuesConsumer 
    * Writes pre-sorted binary docvalues for a field.
    * @param field field information
    * @param values Iterable of binary values in sorted order (deduplicated).
-   * @param docToOrd Iterable of ordinals (one for each document).
+   * @param docToOrd Iterable of ordinals (one for each document). {@code -1} indicates
+   *                 a missing value.
    * @throws IOException if an I/O error occurred.
    */
   public abstract void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException;
@@ -95,7 +98,8 @@ public abstract class DocValuesConsumer 
    * Writes pre-sorted set docvalues for a field
    * @param field field information
    * @param values Iterable of binary values in sorted order (deduplicated).
-   * @param docToOrdCount Iterable of the number of values for each document. 
+   * @param docToOrdCount Iterable of the number of values for each document. A zero ordinal
+   *                      count indicates a missing value.
    * @param ords Iterable of ordinal occurrences (docToOrdCount*maxDoc total).
    * @throws IOException if an I/O error occurred.
    */
@@ -107,7 +111,7 @@ public abstract class DocValuesConsumer 
    * The default implementation calls {@link #addNumericField}, passing
    * an Iterable that merges and filters deleted documents on the fly.
    */
-  public void mergeNumericField(FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge) throws IOException {
+  public void mergeNumericField(final FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
 
     addNumericField(fieldInfo,
                     new Iterable<Number>() {
@@ -116,10 +120,11 @@ public abstract class DocValuesConsumer 
                         return new Iterator<Number>() {
                           int readerUpto = -1;
                           int docIDUpto;
-                          long nextValue;
+                          Long nextValue;
                           AtomicReader currentReader;
                           NumericDocValues currentValues;
                           Bits currentLiveDocs;
+                          Bits currentDocsWithField;
                           boolean nextIsSet;
 
                           @Override
@@ -139,7 +144,6 @@ public abstract class DocValuesConsumer 
                             }
                             assert nextIsSet;
                             nextIsSet = false;
-                            // TODO: make a mutable number
                             return nextValue;
                           }
 
@@ -155,6 +159,7 @@ public abstract class DocValuesConsumer 
                                   currentReader = mergeState.readers.get(readerUpto);
                                   currentValues = toMerge.get(readerUpto);
                                   currentLiveDocs = currentReader.getLiveDocs();
+                                  currentDocsWithField = docsWithField.get(readerUpto);
                                 }
                                 docIDUpto = 0;
                                 continue;
@@ -162,7 +167,11 @@ public abstract class DocValuesConsumer 
 
                               if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
                                 nextIsSet = true;
-                                nextValue = currentValues.get(docIDUpto);
+                                if (currentDocsWithField.get(docIDUpto)) {
+                                  nextValue = currentValues.get(docIDUpto);
+                                } else {
+                                  nextValue = null;
+                                }
                                 docIDUpto++;
                                 return true;
                               }
@@ -181,7 +190,7 @@ public abstract class DocValuesConsumer 
    * The default implementation calls {@link #addBinaryField}, passing
    * an Iterable that merges and filters deleted documents on the fly.
    */
-  public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge) throws IOException {
+  public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge, final List<Bits> docsWithField) throws IOException {
 
     addBinaryField(fieldInfo,
                    new Iterable<BytesRef>() {
@@ -191,9 +200,11 @@ public abstract class DocValuesConsumer 
                          int readerUpto = -1;
                          int docIDUpto;
                          BytesRef nextValue = new BytesRef();
+                         BytesRef nextPointer; // points to null if missing, or nextValue
                          AtomicReader currentReader;
                          BinaryDocValues currentValues;
                          Bits currentLiveDocs;
+                         Bits currentDocsWithField;
                          boolean nextIsSet;
 
                          @Override
@@ -213,8 +224,7 @@ public abstract class DocValuesConsumer 
                            }
                            assert nextIsSet;
                            nextIsSet = false;
-                           // TODO: make a mutable number
-                           return nextValue;
+                           return nextPointer;
                          }
 
                          private boolean setNext() {
@@ -228,6 +238,7 @@ public abstract class DocValuesConsumer 
                                if (readerUpto < toMerge.size()) {
                                  currentReader = mergeState.readers.get(readerUpto);
                                  currentValues = toMerge.get(readerUpto);
+                                 currentDocsWithField = docsWithField.get(readerUpto);
                                  currentLiveDocs = currentReader.getLiveDocs();
                                }
                                docIDUpto = 0;
@@ -236,7 +247,12 @@ public abstract class DocValuesConsumer 
 
                              if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
                                nextIsSet = true;
-                               currentValues.get(docIDUpto, nextValue);
+                               if (currentDocsWithField.get(docIDUpto)) {
+                                 currentValues.get(docIDUpto, nextValue);
+                                 nextPointer = nextValue;
+                               } else {
+                                 nextPointer = null;
+                               }
                                docIDUpto++;
                                return true;
                              }
@@ -272,7 +288,10 @@ public abstract class DocValuesConsumer 
         OpenBitSet bitset = new OpenBitSet(dv.getValueCount());
         for (int i = 0; i < reader.maxDoc(); i++) {
           if (liveDocs.get(i)) {
-            bitset.set(dv.getOrd(i));
+            int ord = dv.getOrd(i);
+            if (ord >= 0) {
+              bitset.set(ord);
+            }
           }
         }
         liveTerms[sub] = new BitsFilteredTermsEnum(dv.termsEnum(), bitset);
@@ -368,7 +387,7 @@ public abstract class DocValuesConsumer 
                   if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
                     nextIsSet = true;
                     int segOrd = dvs[readerUpto].getOrd(docIDUpto);
-                    nextValue = (int) map.getGlobalOrd(readerUpto, segOrd);
+                    nextValue = segOrd == -1 ? -1 : (int) map.getGlobalOrd(readerUpto, segOrd);
                     docIDUpto++;
                     return true;
                   }

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java Fri Aug 16 10:25:38 2013
@@ -25,6 +25,7 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.Bits;
 
 /** Abstract API that produces numeric, binary and
  * sorted docvalues.
@@ -56,4 +57,65 @@ public abstract class DocValuesProducer 
    *  The returned instance need not be thread-safe: it will only be
    *  used by a single thread. */
   public abstract SortedSetDocValues getSortedSet(FieldInfo 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.
+   *  The returned instance need not be thread-safe: it will only be
+   *  used by a single thread. */
+  public abstract Bits getDocsWithField(FieldInfo field) throws IOException;
+  
+  /** 
+   * A simple implementation of {@link DocValuesProducer#getDocsWithField} that 
+   * returns {@code true} if a document has an ordinal &gt;= 0
+   * <p>
+   * Codecs can choose to use this (or implement it more efficiently another way), but
+   * in most cases a Bits is unnecessary anyway: users can check this as they go.
+   */
+  public static class SortedDocsWithField implements Bits {
+    final SortedDocValues in;
+    final int maxDoc;
+    
+    public SortedDocsWithField(SortedDocValues in, int maxDoc) {
+      this.in = in;
+      this.maxDoc = maxDoc;
+    }
+    
+    @Override
+    public boolean get(int index) {
+      return in.getOrd(index) >= 0;
+    }
+
+    @Override
+    public int length() {
+      return maxDoc;
+    }
+  }
+  
+  /** 
+   * A simple implementation of {@link DocValuesProducer#getDocsWithField} that 
+   * returns {@code true} if a document has any ordinals.
+   * <p>
+   * Codecs can choose to use this (or implement it more efficiently another way), but
+   * in most cases a Bits is unnecessary anyway: users can check this as they go.
+   */
+  public static class SortedSetDocsWithField implements Bits {
+    final SortedSetDocValues in;
+    final int maxDoc;
+    
+    public SortedSetDocsWithField(SortedSetDocValues in, int maxDoc) {
+      this.in = in;
+      this.maxDoc = maxDoc;
+    }
+    
+    @Override
+    public boolean get(int index) {
+      in.setDocument(index);
+      return in.nextOrd() != SortedSetDocValues.NO_MORE_ORDS;
+    }
+
+    @Override
+    public int length() {
+      return maxDoc;
+    }
+  }
 }

Added: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/MissingOrdRemapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/MissingOrdRemapper.java?rev=1514642&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/MissingOrdRemapper.java (added)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/MissingOrdRemapper.java Fri Aug 16 10:25:38 2013
@@ -0,0 +1,124 @@
+package org.apache.lucene.codecs;
+
+import java.util.Iterator;
+
+import org.apache.lucene.util.BytesRef;
+
+/*
+ * 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.
+ */
+
+/** 
+ * a utility class to write missing values for SORTED_SET as if they were the empty string
+ * (to simulate pre-Lucene4.5 dv behavior for testing old codecs)
+ */
+// nocommit: move this to test-framework with all the impersonators of
+// these old codecs once new memory/disk codecs are written that support missing
+public class MissingOrdRemapper {
+  
+  /** insert an empty byte[] to the front of this iterable */
+  public static Iterable<BytesRef> insertEmptyValue(final Iterable<BytesRef> iterable) {
+    return new Iterable<BytesRef>() {
+      @Override
+      public Iterator<BytesRef> iterator() {
+        return new Iterator<BytesRef>() {
+          boolean seenEmpty = false;
+          Iterator<BytesRef> in = iterable.iterator();
+          
+          @Override
+          public boolean hasNext() {
+            return !seenEmpty || in.hasNext();
+          }
+
+          @Override
+          public BytesRef next() {
+            if (!seenEmpty) {
+              seenEmpty = true;
+              return new BytesRef();
+            } else {
+              return in.next();
+            }
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+  }
+  
+  /** remaps ord -1 to ord 0 on this iterable. */
+  public static Iterable<Number> mapMissingToOrd0(final Iterable<Number> iterable) {
+    return new Iterable<Number>() {
+      @Override
+      public Iterator<Number> iterator() {
+        return new Iterator<Number>() {
+          Iterator<Number> in = iterable.iterator();
+          
+          @Override
+          public boolean hasNext() {
+            return in.hasNext();
+          }
+
+          @Override
+          public Number next() {
+            Number n = in.next();
+            if (n.longValue() == -1) {
+              return 0;
+            } else {
+              return n;
+            }
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+  }
+  
+  /** remaps every ord+1 on this iterable */
+  public static Iterable<Number> mapAllOrds(final Iterable<Number> iterable) {
+    return new Iterable<Number>() {
+      @Override
+      public Iterator<Number> iterator() {
+        return new Iterator<Number>() {
+          Iterator<Number> in = iterable.iterator();
+          
+          @Override
+          public boolean hasNext() {
+            return in.hasNext();
+          }
+
+          @Override
+          public Number next() {
+            Number n = in.next();
+            return n.longValue()+1;
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+    };
+  }
+}

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java Fri Aug 16 10:25:38 2013
@@ -35,6 +35,7 @@ import org.apache.lucene.index.SortedSet
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
@@ -621,6 +622,11 @@ final class Lucene40DocValuesReader exte
   }
 
   @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    return new Bits.MatchAllBits(state.segmentInfo.getDocCount());
+  }
+
+  @Override
   public void close() throws IOException {
     dir.close();
   }

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java Fri Aug 16 10:25:38 2013
@@ -25,6 +25,7 @@ import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.MissingOrdRemapper;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
@@ -106,7 +107,8 @@ class Lucene42DocValuesConsumer extends 
 
       long count = 0;
       for (Number nv : values) {
-        final long v = nv.longValue();
+        // TODO: support this as MemoryDVFormat (and be smart about missing maybe)
+        final long v = nv == null ? 0 : nv.longValue();
 
         if (gcd != 1) {
           if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
@@ -142,7 +144,7 @@ class Lucene42DocValuesConsumer extends 
       if (formatAndBits.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
         meta.writeByte(UNCOMPRESSED); // uncompressed
         for (Number nv : values) {
-          data.writeByte((byte) nv.longValue());
+          data.writeByte(nv == null ? 0 : (byte) nv.longValue());
         }
       } else {
         meta.writeByte(TABLE_COMPRESSED); // table-compressed
@@ -160,7 +162,7 @@ class Lucene42DocValuesConsumer extends 
 
         final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, maxDoc, formatAndBits.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
         for(Number nv : values) {
-          writer.add(encode.get(nv.longValue()));
+          writer.add(encode.get(nv == null ? 0 : nv.longValue()));
         }
         writer.finish();
       }
@@ -173,7 +175,8 @@ class Lucene42DocValuesConsumer extends 
 
       final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
       for (Number nv : values) {
-        writer.add((nv.longValue() - minValue) / gcd);
+        long value = nv == null ? 0 : nv.longValue();
+        writer.add((value - minValue) / gcd);
       }
       writer.finish();
     } else {
@@ -184,7 +187,7 @@ class Lucene42DocValuesConsumer extends 
 
       final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
       for (Number nv : values) {
-        writer.add(nv.longValue());
+        writer.add(nv == null ? 0 : nv.longValue());
       }
       writer.finish();
     }
@@ -216,9 +219,12 @@ class Lucene42DocValuesConsumer extends 
     int maxLength = Integer.MIN_VALUE;
     final long startFP = data.getFilePointer();
     for(BytesRef v : values) {
-      minLength = Math.min(minLength, v.length);
-      maxLength = Math.max(maxLength, v.length);
-      data.writeBytes(v.bytes, v.offset, v.length);
+      final int length = v == null ? 0 : v.length;
+      minLength = Math.min(minLength, length);
+      maxLength = Math.max(maxLength, length);
+      if (v != null) {
+        data.writeBytes(v.bytes, v.offset, v.length);
+      }
     }
     meta.writeLong(startFP);
     meta.writeLong(data.getFilePointer() - startFP);
@@ -234,7 +240,9 @@ class Lucene42DocValuesConsumer extends 
       final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
       long addr = 0;
       for (BytesRef v : values) {
-        addr += v.length;
+        if (v != null) {
+          addr += v.length;
+        }
         writer.add(addr);
       }
       writer.finish();
@@ -262,6 +270,33 @@ class Lucene42DocValuesConsumer extends 
 
   @Override
   public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    // three cases for simulating the old writer:
+    // 1. no missing
+    // 2. missing (and empty string in use): remap ord=-1 -> ord=0
+    // 3. missing (and empty string not in use): remap all ords +1, insert empty string into values
+    boolean anyMissing = false;
+    for (Number n : docToOrd) {
+      if (n.longValue() == -1) {
+        anyMissing = true;
+        break;
+      }
+    }
+    
+    boolean hasEmptyString = false;
+    for (BytesRef b : values) {
+      hasEmptyString = b.length == 0;
+      break;
+    }
+    
+    if (!anyMissing) {
+      // nothing to do
+    } else if (hasEmptyString) {
+      docToOrd = MissingOrdRemapper.mapMissingToOrd0(docToOrd);
+    } else {
+      docToOrd = MissingOrdRemapper.mapAllOrds(docToOrd);
+      values = MissingOrdRemapper.insertEmptyValue(values);
+    }
+    
     // write the ordinals as numerics
     addNumericField(field, docToOrd, false);
     

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Fri Aug 16 10:25:38 2013
@@ -429,6 +429,15 @@ class Lucene42DocValuesProducer extends 
       }
     };
   }
+  
+  @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
+      return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
+    } else {
+      return new Bits.MatchAllBits(maxDoc);
+    }
+  }
 
   @Override
   public void close() throws IOException {

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java Fri Aug 16 10:25:38 2013
@@ -26,6 +26,8 @@ import org.apache.lucene.util.ByteBlockP
 import org.apache.lucene.util.ByteBlockPool;
 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.packed.AppendingDeltaPackedLongBuffer;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -38,6 +40,9 @@ class BinaryDocValuesWriter extends DocV
 
   private final ByteBlockPool pool;
   private final AppendingDeltaPackedLongBuffer lengths;
+  private final OpenBitSet docsWithField;
+  private final Counter iwBytesUsed;
+  private long bytesUsed;
   private final FieldInfo fieldInfo;
   private int addedValues = 0;
 
@@ -45,6 +50,10 @@ class BinaryDocValuesWriter extends DocV
     this.fieldInfo = fieldInfo;
     this.pool = new ByteBlockPool(new DirectTrackingAllocator(iwBytesUsed));
     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) {
@@ -66,6 +75,19 @@ class BinaryDocValuesWriter extends DocV
     addedValues++;
     lengths.add(value.length);
     pool.append(value);
+    docsWithField.set(docID);
+    updateBytesUsed();
+  }
+  
+  private long docsWithFieldBytesUsed() {
+    // nocommit: this is not correct
+    return docsWithField.getBits().length*RamUsageEstimator.NUM_BYTES_LONG;
+  }
+  
+  private void updateBytesUsed() {
+    final long newBytesUsed = docsWithFieldBytesUsed();
+    iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
+    bytesUsed = newBytesUsed;
   }
 
   @Override
@@ -111,19 +133,23 @@ class BinaryDocValuesWriter extends DocV
       if (!hasNext()) {
         throw new NoSuchElementException();
       }
+      final BytesRef v;
       if (upto < size) {
         int length = (int) lengthsIterator.next();
         value.grow(length);
         value.length = length;
         pool.readBytes(byteOffset, value.bytes, value.offset, value.length);
         byteOffset += length;
+        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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java Fri Aug 16 10:25:38 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() {
+    // nocommit: this is not correct
+    return docsWithField.getBits().length*RamUsageEstimator.NUM_BYTES_LONG;
+  }
 
   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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Fri Aug 16 10:25:38 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/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java?rev=1514642&r1=1514641&r2=1514642&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/SortedDocValues.java Fri Aug 16 10:25:38 2013
@@ -37,7 +37,8 @@ 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);
 
@@ -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;
     }
   };