You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/08/30 17:06:49 UTC

svn commit: r1518989 [2/10] - in /lucene/dev/branches/lucene3069: ./ dev-tools/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/solr/core/src/java/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/...

Modified: lucene/dev/branches/lucene3069/lucene/analysis/uima/src/java/org/apache/lucene/analysis/uima/UIMAAnnotationsTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/analysis/uima/src/java/org/apache/lucene/analysis/uima/UIMAAnnotationsTokenizer.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/analysis/uima/src/java/org/apache/lucene/analysis/uima/UIMAAnnotationsTokenizer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/analysis/uima/src/java/org/apache/lucene/analysis/uima/UIMAAnnotationsTokenizer.java Fri Aug 30 15:06:42 2013
@@ -86,7 +86,7 @@ public final class UIMAAnnotationsTokeni
 
   @Override
   public void end() throws IOException {
-    offsetAttr.setOffset(finalOffset, finalOffset);
     super.end();
+    offsetAttr.setOffset(finalOffset, finalOffset);
   }
 }

Modified: lucene/dev/branches/lucene3069/lucene/analysis/uima/src/java/org/apache/lucene/analysis/uima/UIMATypeAwareAnnotationsTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/analysis/uima/src/java/org/apache/lucene/analysis/uima/UIMATypeAwareAnnotationsTokenizer.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/analysis/uima/src/java/org/apache/lucene/analysis/uima/UIMATypeAwareAnnotationsTokenizer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/analysis/uima/src/java/org/apache/lucene/analysis/uima/UIMATypeAwareAnnotationsTokenizer.java Fri Aug 30 15:06:42 2013
@@ -107,8 +107,8 @@ public final class UIMATypeAwareAnnotati
 
   @Override
   public void end() throws IOException {
-    offsetAttr.setOffset(finalOffset, finalOffset);
     super.end();
+    offsetAttr.setOffset(finalOffset, finalOffset);
   }
 
 

Modified: lucene/dev/branches/lucene3069/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (original)
+++ lucene/dev/branches/lucene3069/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java Fri Aug 30 15:06:42 2013
@@ -342,7 +342,7 @@ public class TestPerfTasksLogic extends 
     Benchmark benchmark = execBenchmark(algLines);
 
     DirectoryReader r = DirectoryReader.open(benchmark.getRunData().getDirectory());
-    SortedDocValues idx = FieldCache.DEFAULT.getTermsIndex(new SlowCompositeReaderWrapper(r), "country");
+    SortedDocValues idx = FieldCache.DEFAULT.getTermsIndex(SlowCompositeReaderWrapper.wrap(r), "country");
     final int maxDoc = r.maxDoc();
     assertEquals(1000, maxDoc);
     for(int i=0;i<1000;i++) {

Modified: lucene/dev/branches/lucene3069/lucene/classification/src/test/org/apache/lucene/classification/ClassificationTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/classification/src/test/org/apache/lucene/classification/ClassificationTestBase.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/classification/src/test/org/apache/lucene/classification/ClassificationTestBase.java (original)
+++ lucene/dev/branches/lucene3069/lucene/classification/src/test/org/apache/lucene/classification/ClassificationTestBase.java Fri Aug 30 15:06:42 2013
@@ -21,6 +21,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.store.Directory;
@@ -66,10 +67,10 @@ public abstract class ClassificationTest
 
 
   protected void checkCorrectClassification(Classifier<T> classifier, String inputDoc, T expectedResult, Analyzer analyzer, String classFieldName) throws Exception {
-    SlowCompositeReaderWrapper compositeReaderWrapper = null;
+    AtomicReader compositeReaderWrapper = null;
     try {
       populateIndex(analyzer);
-      compositeReaderWrapper = new SlowCompositeReaderWrapper(indexWriter.getReader());
+      compositeReaderWrapper = SlowCompositeReaderWrapper.wrap(indexWriter.getReader());
       classifier.train(compositeReaderWrapper, textFieldName, classFieldName, analyzer);
       ClassificationResult<T> classificationResult = classifier.assignClass(inputDoc);
       assertNotNull(classificationResult.getAssignedClass());

Modified: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java Fri Aug 30 15:06:42 2013
@@ -22,8 +22,11 @@ import java.io.IOException;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.BytesRef;
 
 /**
  * DocValues format that keeps most things on disk.
@@ -40,7 +43,12 @@ public final class DiskDocValuesFormat e
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    return new DiskDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+    return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION) {
+      @Override
+      protected void addTermsDict(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+        addBinaryField(field, values);
+      }
+    };
   }
 
   @Override
@@ -52,11 +60,4 @@ public final class DiskDocValuesFormat e
   public static final String DATA_EXTENSION = "dvdd";
   public static final String META_CODEC = "DiskDocValuesMetadata";
   public static final String META_EXTENSION = "dvdm";
-  public static final int VERSION_START = 0;
-  public static final int VERSION_COMPRESSED_TERMS = 1;
-  public static final int VERSION_CURRENT = VERSION_COMPRESSED_TERMS;
-  public static final byte NUMERIC = 0;
-  public static final byte BINARY = 1;
-  public static final byte SORTED = 2;
-  public static final byte SORTED_SET = 3;
 }

Modified: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java Fri Aug 30 15:06:42 2013
@@ -17,726 +17,34 @@ package org.apache.lucene.codecs.diskdv;
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.DELTA_COMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.GCD_COMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.TABLE_COMPRESSED;
-
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_PREFIX_COMPRESSED;
-
 import java.io.IOException;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Map;
 
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.codecs.lucene45.Lucene45DocValuesProducer;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
 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.packed.BlockPackedReader;
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
-import org.apache.lucene.util.packed.PackedInts;
 
-class DiskDocValuesProducer extends DocValuesProducer {
-  private final Map<Integer,NumericEntry> numerics;
-  private final Map<Integer,BinaryEntry> binaries;
-  private final Map<Integer,NumericEntry> ords;
-  private final Map<Integer,NumericEntry> ordIndexes;
-  private final IndexInput data;
+class DiskDocValuesProducer extends Lucene45DocValuesProducer {
 
-  // memory-resident structures
-  private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
-  private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
-  
   DiskDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
-    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);
-    boolean success = false;
-    final int version;
-    try {
-      version = CodecUtil.checkHeader(in, metaCodec, 
-                                      DiskDocValuesFormat.VERSION_CURRENT,
-                                      DiskDocValuesFormat.VERSION_CURRENT);
-      numerics = new HashMap<Integer,NumericEntry>();
-      ords = new HashMap<Integer,NumericEntry>();
-      ordIndexes = new HashMap<Integer,NumericEntry>();
-      binaries = new HashMap<Integer,BinaryEntry>();
-      readFields(in, state.fieldInfos);
-
-      success = true;
-    } finally {
-      if (success) {
-        IOUtils.close(in);
-      } else {
-        IOUtils.closeWhileHandlingException(in);
-      }
-    }
-
-    success = false;
-    try {
-      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
-      data = state.directory.openInput(dataName, state.context);
-      final int version2 = CodecUtil.checkHeader(data, dataCodec, 
-                                                 DiskDocValuesFormat.VERSION_CURRENT,
-                                                 DiskDocValuesFormat.VERSION_CURRENT);
-      if (version != version2) {
-        throw new CorruptIndexException("Format versions mismatch");
-      }
-
-      success = true;
-    } finally {
-      if (!success) {
-        IOUtils.closeWhileHandlingException(this.data);
-      }
-    }
-  }
-  
-  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
-    int fieldNumber = meta.readVInt();
-    while (fieldNumber != -1) {
-      byte type = meta.readByte();
-      if (type == DiskDocValuesFormat.NUMERIC) {
-        numerics.put(fieldNumber, readNumericEntry(meta));
-      } else if (type == DiskDocValuesFormat.BINARY) {
-        BinaryEntry b = readBinaryEntry(meta);
-        binaries.put(fieldNumber, b);
-      } else if (type == DiskDocValuesFormat.SORTED) {
-        // sorted = binary + numeric
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != DiskDocValuesFormat.BINARY) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        BinaryEntry b = readBinaryEntry(meta);
-        binaries.put(fieldNumber, b);
-        
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        NumericEntry n = readNumericEntry(meta);
-        ords.put(fieldNumber, n);
-      } else if (type == DiskDocValuesFormat.SORTED_SET) {
-        // sortedset = binary + numeric + ordIndex
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != DiskDocValuesFormat.BINARY) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        BinaryEntry b = readBinaryEntry(meta);
-        binaries.put(fieldNumber, b);
-        
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        NumericEntry n1 = readNumericEntry(meta);
-        ords.put(fieldNumber, n1);
-        
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        NumericEntry n2 = readNumericEntry(meta);
-        ordIndexes.put(fieldNumber, n2);
-      } else {
-        throw new CorruptIndexException("invalid type: " + type + ", resource=" + meta);
-      }
-      fieldNumber = meta.readVInt();
-    }
-  }
-  
-  static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
-    NumericEntry entry = new NumericEntry();
-    entry.format = meta.readVInt();
-    entry.packedIntsVersion = meta.readVInt();
-    entry.offset = meta.readLong();
-    entry.count = meta.readVLong();
-    entry.blockSize = meta.readVInt();
-    switch(entry.format) {
-      case GCD_COMPRESSED:
-        entry.minValue = meta.readLong();
-        entry.gcd = meta.readLong();
-        break;
-      case TABLE_COMPRESSED:
-        if (entry.count > Integer.MAX_VALUE) {
-          throw new CorruptIndexException("Cannot use TABLE_COMPRESSED with more than MAX_VALUE values, input=" + meta);
-        }
-        final int uniqueValues = meta.readVInt();
-        if (uniqueValues > 256) {
-          throw new CorruptIndexException("TABLE_COMPRESSED cannot have more than 256 distinct values, input=" + meta);
-        }
-        entry.table = new long[uniqueValues];
-        for (int i = 0; i < uniqueValues; ++i) {
-          entry.table[i] = meta.readLong();
-        }
-        break;
-      case DELTA_COMPRESSED:
-        break;
-      default:
-        throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
-    }
-    return entry;
-  }
-  
-  static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
-    BinaryEntry entry = new BinaryEntry();
-    entry.format = meta.readVInt();
-    entry.minLength = meta.readVInt();
-    entry.maxLength = meta.readVInt();
-    entry.count = meta.readVLong();
-    entry.offset = meta.readLong();
-    switch(entry.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        break;
-      case BINARY_PREFIX_COMPRESSED:
-        entry.addressInterval = meta.readVInt();
-        entry.addressesOffset = meta.readLong();
-        entry.packedIntsVersion = meta.readVInt();
-        entry.blockSize = meta.readVInt();
-        break;
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        entry.addressesOffset = meta.readLong();
-        entry.packedIntsVersion = meta.readVInt();
-        entry.blockSize = meta.readVInt();
-        break;
-      default:
-        throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
-    }
-    return entry;
+    super(state, dataCodec, dataExtension, metaCodec, metaExtension);
   }
 
   @Override
-  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
-    NumericEntry entry = numerics.get(field.number);
-    return getNumeric(entry);
-  }
-  
-  LongNumericDocValues getNumeric(NumericEntry entry) throws IOException {
-    final IndexInput data = this.data.clone();
-    data.seek(entry.offset);
-
-    switch (entry.format) {
-      case DELTA_COMPRESSED:
-        final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
-        return new LongNumericDocValues() {
-          @Override
-          public long get(long id) {
-            return reader.get(id);
-          }
-        };
-      case GCD_COMPRESSED:
-        final long min = entry.minValue;
-        final long mult = entry.gcd;
-        final BlockPackedReader quotientReader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
-        return new LongNumericDocValues() {
-          @Override
-          public long get(long id) {
-            return min + mult * quotientReader.get(id);
-          }
-        };
-      case TABLE_COMPRESSED:
-        final long table[] = entry.table;
-        final int bitsRequired = PackedInts.bitsRequired(table.length - 1);
-        final PackedInts.Reader ords = PackedInts.getDirectReaderNoHeader(data, PackedInts.Format.PACKED, entry.packedIntsVersion, (int) entry.count, bitsRequired);
-        return new LongNumericDocValues() {
-          @Override
-          public long get(long id) {
-            return table[(int) ords.get((int) id)];
-          }
-        };
-      default:
-        throw new AssertionError();
-    }
+  protected MonotonicBlockPackedReader getAddressInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
+    data.seek(bytes.addressesOffset);
+    return new MonotonicBlockPackedReader(data.clone(), bytes.packedIntsVersion, bytes.blockSize, bytes.count, true);
   }
 
   @Override
-  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
-    BinaryEntry bytes = binaries.get(field.number);
-    switch(bytes.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        return getFixedBinary(field, bytes);
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        return getVariableBinary(field, bytes);
-      case BINARY_PREFIX_COMPRESSED:
-        return getCompressedBinary(field, bytes);
-      default:
-        throw new AssertionError();
-    }
-  }
-  
-  private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) {
-    final IndexInput data = this.data.clone();
-
-    return new LongBinaryDocValues() {
-      @Override
-      public void get(long id, BytesRef result) {
-        long address = bytes.offset + id * bytes.maxLength;
-        try {
-          data.seek(address);
-          // NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource) 
-          // assume "they" own the bytes after calling this!
-          final byte[] buffer = new byte[bytes.maxLength];
-          data.readBytes(buffer, 0, buffer.length);
-          result.bytes = buffer;
-          result.offset = 0;
-          result.length = buffer.length;
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-  }
-  
-  private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final IndexInput data = this.data.clone();
-    
-    final MonotonicBlockPackedReader addresses;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
-        addressInstances.put(field.number, addrInstance);
-      }
-      addresses = addrInstance;
-    }
-
-    return new LongBinaryDocValues() {
-      @Override
-      public void get(long id, BytesRef result) {
-        long startAddress = bytes.offset + (id == 0 ? 0 : addresses.get(id-1));
-        long endAddress = bytes.offset + addresses.get(id);
-        int length = (int) (endAddress - startAddress);
-        try {
-          data.seek(startAddress);
-          // NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource) 
-          // assume "they" own the bytes after calling this!
-          final byte[] buffer = new byte[length];
-          data.readBytes(buffer, 0, buffer.length);
-          result.bytes = buffer;
-          result.offset = 0;
-          result.length = length;
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-  }
-
-  private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final IndexInput data = this.data.clone();
-    final long interval = bytes.addressInterval;
-
-    final MonotonicBlockPackedReader addresses;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        final long size;
-        if (bytes.count % interval == 0) {
-          size = bytes.count / interval;
-        } else {
-          size = 1L + bytes.count / interval;
-        }
-        addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-        addressInstances.put(field.number, addrInstance);
-      }
-      addresses = addrInstance;
-    }
-    
-    return new CompressedBinaryDocValues(bytes, addresses, data);
+  protected MonotonicBlockPackedReader getIntervalInstance(IndexInput data, FieldInfo field, BinaryEntry bytes) throws IOException {
+    throw new AssertionError();
   }
 
   @Override
-  public SortedDocValues getSorted(FieldInfo field) throws IOException {
-    final int valueCount = (int) binaries.get(field.number).count;
-    final BinaryDocValues binary = getBinary(field);
-    NumericEntry entry = ords.get(field.number);
-    IndexInput data = this.data.clone();
+  protected MonotonicBlockPackedReader getOrdIndexInstance(IndexInput data, FieldInfo field, NumericEntry entry) throws IOException {
     data.seek(entry.offset);
-    final BlockPackedReader ordinals = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
-    
-    return new SortedDocValues() {
-
-      @Override
-      public int getOrd(int docID) {
-        return (int) ordinals.get(docID);
-      }
-
-      @Override
-      public void lookupOrd(int ord, BytesRef result) {
-        binary.get(ord, result);
-      }
-
-      @Override
-      public int getValueCount() {
-        return valueCount;
-      }
-
-      @Override
-      public int lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-        return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
-    };
-  }
-
-  @Override
-  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
-    final long valueCount = binaries.get(field.number).count;
-    // we keep the byte[]s and list of ords on disk, these could be large
-    final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
-    final LongNumericDocValues ordinals = getNumeric(ords.get(field.number));
-    // but the addresses to the ord stream are in RAM
-    final MonotonicBlockPackedReader ordIndex;
-    synchronized (ordIndexInstances) {
-      MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
-      if (ordIndexInstance == null) {
-        NumericEntry entry = ordIndexes.get(field.number);
-        IndexInput data = this.data.clone();
-        data.seek(entry.offset);
-        ordIndexInstance = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
-        ordIndexInstances.put(field.number, ordIndexInstance);
-      }
-      ordIndex = ordIndexInstance;
-    }
-    
-    return new SortedSetDocValues() {
-      long offset;
-      long endOffset;
-      
-      @Override
-      public long nextOrd() {
-        if (offset == endOffset) {
-          return NO_MORE_ORDS;
-        } else {
-          long ord = ordinals.get(offset);
-          offset++;
-          return ord;
-        }
-      }
-
-      @Override
-      public void setDocument(int docID) {
-        offset = (docID == 0 ? 0 : ordIndex.get(docID-1));
-        endOffset = ordIndex.get(docID);
-      }
-
-      @Override
-      public void lookupOrd(long ord, BytesRef result) {
-        binary.get(ord, result);
-      }
-
-      @Override
-      public long getValueCount() {
-        return valueCount;
-      }
-      
-      @Override
-      public long lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
-    };
-  }
-
-  @Override
-  public void close() throws IOException {
-    data.close();
-  }
-  
-  static class NumericEntry {
-    long offset;
-
-    int format;
-    int packedIntsVersion;
-    long count;
-    int blockSize;
-    
-    long minValue;
-    long gcd;
-    long table[];
-  }
-  
-  static class BinaryEntry {
-    long offset;
-
-    int format;
-    long count;
-    int minLength;
-    int maxLength;
-    long addressesOffset;
-    long addressInterval;
-    int packedIntsVersion;
-    int blockSize;
-  }
-  
-  // internally we compose complex dv (sorted/sortedset) from other ones
-  static abstract class LongNumericDocValues extends NumericDocValues {
-    @Override
-    public final long get(int docID) {
-      return get((long) docID);
-    }
-    
-    abstract long get(long id);
-  }
-  
-  static abstract class LongBinaryDocValues extends BinaryDocValues {
-    @Override
-    public final void get(int docID, BytesRef result) {
-      get((long)docID, result);
-    }
-    
-    abstract void get(long id, BytesRef Result);
-  }
-  
-  // in the compressed case, we add a few additional operations for
-  // more efficient reverse lookup and enumeration
-  static class CompressedBinaryDocValues extends LongBinaryDocValues {
-    final BinaryEntry bytes;
-    final long interval;
-    final long numValues;
-    final long numIndexValues;
-    final MonotonicBlockPackedReader addresses;
-    final IndexInput data;
-    final TermsEnum termsEnum;
-    
-    public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, IndexInput data) throws IOException {
-      this.bytes = bytes;
-      this.interval = bytes.addressInterval;
-      this.addresses = addresses;
-      this.data = data;
-      this.numValues = bytes.count;
-      this.numIndexValues = addresses.size();
-      this.termsEnum = getTermsEnum(data);
-    }
-    
-    @Override
-    public void get(long id, BytesRef result) {
-      try {
-        termsEnum.seekExact(id);
-        BytesRef term = termsEnum.term();
-        result.bytes = term.bytes;
-        result.offset = term.offset;
-        result.length = term.length;
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    
-    long lookupTerm(BytesRef key) {
-      try {
-        SeekStatus status = termsEnum.seekCeil(key);
-        if (status == SeekStatus.END) {
-          return -numValues-1;
-        } else if (status == SeekStatus.FOUND) {
-          return termsEnum.ord();
-        } else {
-          return -termsEnum.ord()-1;
-        }
-      } catch (IOException bogus) {
-        throw new RuntimeException(bogus);
-      }
-    }
-    
-    TermsEnum getTermsEnum() {
-      try {
-        return getTermsEnum(data.clone());
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    
-    private TermsEnum getTermsEnum(final IndexInput input) throws IOException {
-      input.seek(bytes.offset);
-      
-      return new TermsEnum() {
-        private long currentOrd = -1;
-        // TODO: maxLength is negative when all terms are merged away...
-        private final BytesRef termBuffer = new BytesRef(bytes.maxLength < 0 ? 0 : bytes.maxLength);
-        private final BytesRef term = new BytesRef(); // TODO: paranoia?
-
-        @Override
-        public BytesRef next() throws IOException {
-          if (doNext() == null) {
-            return null;
-          } else {
-            setTerm();
-            return term;
-          }
-        }
-        
-        private BytesRef doNext() throws IOException {
-          if (++currentOrd >= numValues) {
-            return null;
-          } else {
-            int start = input.readVInt();
-            int suffix = input.readVInt();
-            input.readBytes(termBuffer.bytes, start, suffix);
-            termBuffer.length = start + suffix;
-            return termBuffer;
-          }
-        }
-
-        @Override
-        public SeekStatus seekCeil(BytesRef text) throws IOException {
-          // binary-search just the index values to find the block,
-          // then scan within the block
-          long low = 0;
-          long high = numIndexValues-1;
-
-          while (low <= high) {
-            long mid = (low + high) >>> 1;
-            doSeek(mid * interval);
-            int cmp = termBuffer.compareTo(text);
-
-            if (cmp < 0) {
-              low = mid + 1;
-            } else if (cmp > 0) {
-              high = mid - 1;
-            } else {
-              // we got lucky, found an indexed term
-              setTerm();
-              return SeekStatus.FOUND;
-            }
-          }
-          
-          if (numIndexValues == 0) {
-            return SeekStatus.END;
-          }
-          
-          // block before insertion point
-          long block = low-1;
-          doSeek(block < 0 ? -1 : block * interval);
-          
-          while (doNext() != null) {
-            int cmp = termBuffer.compareTo(text);
-            if (cmp == 0) {
-              setTerm();
-              return SeekStatus.FOUND;
-            } else if (cmp > 0) {
-              setTerm();
-              return SeekStatus.NOT_FOUND;
-            }
-          }
-          
-          return SeekStatus.END;
-        }
-
-        @Override
-        public void seekExact(long ord) throws IOException {
-          doSeek(ord);
-          setTerm();
-        }
-        
-        private void doSeek(long ord) throws IOException {
-          long block = ord / interval;
-
-          if (ord >= currentOrd && block == currentOrd / interval) {
-            // seek within current block
-          } else {
-            // position before start of block
-            currentOrd = ord - ord % interval - 1;
-            input.seek(bytes.offset + addresses.get(block));
-          }
-          
-          while (currentOrd < ord) {
-            doNext();
-          }
-        }
-        
-        private void setTerm() {
-          // TODO: is there a cleaner way
-          term.bytes = new byte[termBuffer.length];
-          term.offset = 0;
-          term.copyBytes(termBuffer);
-        }
-
-        @Override
-        public BytesRef term() throws IOException {
-          return term;
-        }
-
-        @Override
-        public long ord() throws IOException {
-          return currentOrd;
-        }
-        
-        @Override
-        public Comparator<BytesRef> getComparator() {
-          return BytesRef.getUTF8SortedAsUnicodeComparator();
-        }
-
-        @Override
-        public int docFreq() throws IOException {
-          throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public long totalTermFreq() throws IOException {
-          return -1;
-        }
-
-        @Override
-        public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-          throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-          throw new UnsupportedOperationException();
-        }
-      };
-    }
+    return new MonotonicBlockPackedReader(data.clone(), entry.packedIntsVersion, entry.blockSize, entry.count, true);
   }
 }

Modified: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/memory/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/memory/package.html?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/memory/package.html (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/memory/package.html Fri Aug 30 15:06:42 2013
@@ -20,6 +20,6 @@
    <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
 </head>
 <body>
-Postings format that is read entirely into memory.
+Postings and DocValues formats that are read entirely into memory.
 </body>
 </html>
\ No newline at end of file

Modified: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java Fri Aug 30 15:06:42 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/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Fri Aug 30 15:06:42 2013
@@ -38,6 +38,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 +101,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 +110,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 +159,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 +170,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 +178,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 +222,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 +243,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 +306,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);
@@ -362,6 +427,22 @@ class SimpleTextDocValuesReader extends 
       }
     };
   }
+  
+  @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    switch (field.getDocValuesType()) {
+      case SORTED_SET:
+        return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
+      case SORTED:
+        return new SortedDocsWithField(getSorted(field), maxDoc);
+      case BINARY:
+        return getBinaryDocsWithField(field);
+      case NUMERIC:
+        return getNumericDocsWithField(field);
+      default:
+        throw new AssertionError();
+    }
+  }
 
   @Override
   public void close() throws IOException {

Modified: lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java Fri Aug 30 15:06:42 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/lucene3069/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/branches/lucene3069/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Fri Aug 30 15:06:42 2013
@@ -14,4 +14,5 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.diskdv.DiskDocValuesFormat
+org.apache.lucene.codecs.memory.MemoryDocValuesFormat
 org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java Fri Aug 30 15:06:42 2013
@@ -72,12 +72,15 @@ public abstract class Analyzer implement
 
   private final ReuseStrategy reuseStrategy;
 
+  // non final as it gets nulled if closed; pkg private for access by ReuseStrategy's final helper methods:
+  CloseableThreadLocal<Object> storedValue = new CloseableThreadLocal<Object>();
+
   /**
    * Create a new Analyzer, reusing the same set of components per-thread
    * across calls to {@link #tokenStream(String, Reader)}. 
    */
   public Analyzer() {
-    this(new GlobalReuseStrategy());
+    this(GLOBAL_REUSE_STRATEGY);
   }
 
   /**
@@ -133,11 +136,11 @@ public abstract class Analyzer implement
    */
   public final TokenStream tokenStream(final String fieldName,
                                        final Reader reader) throws IOException {
-    TokenStreamComponents components = reuseStrategy.getReusableComponents(fieldName);
+    TokenStreamComponents components = reuseStrategy.getReusableComponents(this, fieldName);
     final Reader r = initReader(fieldName, reader);
     if (components == null) {
       components = createComponents(fieldName, r);
-      reuseStrategy.setReusableComponents(fieldName, components);
+      reuseStrategy.setReusableComponents(this, fieldName, components);
     } else {
       components.setReader(r);
     }
@@ -167,7 +170,7 @@ public abstract class Analyzer implement
    * @see #tokenStream(String, Reader)
    */
   public final TokenStream tokenStream(final String fieldName, final String text) throws IOException {
-    TokenStreamComponents components = reuseStrategy.getReusableComponents(fieldName);
+    TokenStreamComponents components = reuseStrategy.getReusableComponents(this, fieldName);
     @SuppressWarnings("resource") final ReusableStringReader strReader = 
         (components == null || components.reusableStringReader == null) ?
         new ReusableStringReader() : components.reusableStringReader;
@@ -175,7 +178,7 @@ public abstract class Analyzer implement
     final Reader r = initReader(fieldName, strReader);
     if (components == null) {
       components = createComponents(fieldName, r);
-      reuseStrategy.setReusableComponents(fieldName, components);
+      reuseStrategy.setReusableComponents(this, fieldName, components);
     } else {
       components.setReader(r);
     }
@@ -229,10 +232,20 @@ public abstract class Analyzer implement
     return 1;
   }
 
+  /**
+   * Returns the used {@link ReuseStrategy}.
+   */
+  public final ReuseStrategy getReuseStrategy() {
+    return reuseStrategy;
+  }
+
   /** Frees persistent resources used by this Analyzer */
   @Override
   public void close() {
-    reuseStrategy.close();
+    if (storedValue != null) {
+      storedValue.close();
+      storedValue = null;
+    }
   }
 
   /**
@@ -317,126 +330,101 @@ public abstract class Analyzer implement
    * Strategy defining how TokenStreamComponents are reused per call to
    * {@link Analyzer#tokenStream(String, java.io.Reader)}.
    */
-  public static abstract class ReuseStrategy implements Closeable {
-
-    private CloseableThreadLocal<Object> storedValue = new CloseableThreadLocal<Object>();
+  public static abstract class ReuseStrategy {
 
     /** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
     public ReuseStrategy() {}
 
     /**
-     * Gets the reusable TokenStreamComponents for the field with the given name
+     * Gets the reusable TokenStreamComponents for the field with the given name.
      *
+     * @param analyzer Analyzer from which to get the reused components. Use
+     *        {@link #getStoredValue(Analyzer)} and {@link #setStoredValue(Analyzer, Object)}
+     *        to access the data on the Analyzer.
      * @param fieldName Name of the field whose reusable TokenStreamComponents
      *        are to be retrieved
      * @return Reusable TokenStreamComponents for the field, or {@code null}
      *         if there was no previous components for the field
      */
-    public abstract TokenStreamComponents getReusableComponents(String fieldName);
+    public abstract TokenStreamComponents getReusableComponents(Analyzer analyzer, String fieldName);
 
     /**
      * Stores the given TokenStreamComponents as the reusable components for the
-     * field with the give name
+     * field with the give name.
      *
      * @param fieldName Name of the field whose TokenStreamComponents are being set
      * @param components TokenStreamComponents which are to be reused for the field
      */
-    public abstract void setReusableComponents(String fieldName, TokenStreamComponents components);
+    public abstract void setReusableComponents(Analyzer analyzer, String fieldName, TokenStreamComponents components);
 
     /**
-     * Returns the currently stored value
+     * Returns the currently stored value.
      *
      * @return Currently stored value or {@code null} if no value is stored
-     * @throws AlreadyClosedException if the ReuseStrategy is closed.
+     * @throws AlreadyClosedException if the Analyzer is closed.
      */
-    protected final Object getStoredValue() {
-      try {
-        return storedValue.get();
-      } catch (NullPointerException npe) {
-        if (storedValue == null) {
-          throw new AlreadyClosedException("this Analyzer is closed");
-        } else {
-          throw npe;
-        }
+    protected final Object getStoredValue(Analyzer analyzer) {
+      if (analyzer.storedValue == null) {
+        throw new AlreadyClosedException("this Analyzer is closed");
       }
+      return analyzer.storedValue.get();
     }
 
     /**
-     * Sets the stored value
+     * Sets the stored value.
      *
      * @param storedValue Value to store
-     * @throws AlreadyClosedException if the ReuseStrategy is closed.
+     * @throws AlreadyClosedException if the Analyzer is closed.
      */
-    protected final void setStoredValue(Object storedValue) {
-      try {
-        this.storedValue.set(storedValue);
-      } catch (NullPointerException npe) {
-        if (storedValue == null) {
-          throw new AlreadyClosedException("this Analyzer is closed");
-        } else {
-          throw npe;
-        }
+    protected final void setStoredValue(Analyzer analyzer, Object storedValue) {
+      if (analyzer.storedValue == null) {
+        throw new AlreadyClosedException("this Analyzer is closed");
       }
+      analyzer.storedValue.set(storedValue);
     }
 
-    /**
-     * Closes the ReuseStrategy, freeing any resources
-     */
-    @Override
-    public void close() {
-      if (storedValue != null) {
-        storedValue.close();
-        storedValue = null;
-      }
-    }
   }
 
   /**
-   * Implementation of {@link ReuseStrategy} that reuses the same components for
+   * A predefined {@link ReuseStrategy}  that reuses the same components for
    * every field.
    */
-  public final static class GlobalReuseStrategy extends ReuseStrategy {
-    
-    /** Creates a new instance, with empty per-thread values */
-    public GlobalReuseStrategy() {}
+  public static final ReuseStrategy GLOBAL_REUSE_STRATEGY = new ReuseStrategy() {
 
     @Override
-    public TokenStreamComponents getReusableComponents(String fieldName) {
-      return (TokenStreamComponents) getStoredValue();
+    public TokenStreamComponents getReusableComponents(Analyzer analyzer, String fieldName) {
+      return (TokenStreamComponents) getStoredValue(analyzer);
     }
 
     @Override
-    public void setReusableComponents(String fieldName, TokenStreamComponents components) {
-      setStoredValue(components);
+    public void setReusableComponents(Analyzer analyzer, String fieldName, TokenStreamComponents components) {
+      setStoredValue(analyzer, components);
     }
-  }
+  };
 
   /**
-   * Implementation of {@link ReuseStrategy} that reuses components per-field by
+   * A predefined {@link ReuseStrategy} that reuses components per-field by
    * maintaining a Map of TokenStreamComponent per field name.
    */
-  public static class PerFieldReuseStrategy extends ReuseStrategy {
-
-    /** Creates a new instance, with empty per-thread-per-field values */
-    public PerFieldReuseStrategy() {}
+  public static final ReuseStrategy PER_FIELD_REUSE_STRATEGY = new ReuseStrategy() {
 
     @SuppressWarnings("unchecked")
     @Override
-    public TokenStreamComponents getReusableComponents(String fieldName) {
-      Map<String, TokenStreamComponents> componentsPerField = (Map<String, TokenStreamComponents>) getStoredValue();
+    public TokenStreamComponents getReusableComponents(Analyzer analyzer, String fieldName) {
+      Map<String, TokenStreamComponents> componentsPerField = (Map<String, TokenStreamComponents>) getStoredValue(analyzer);
       return componentsPerField != null ? componentsPerField.get(fieldName) : null;
     }
 
     @SuppressWarnings("unchecked")
     @Override
-    public void setReusableComponents(String fieldName, TokenStreamComponents components) {
-      Map<String, TokenStreamComponents> componentsPerField = (Map<String, TokenStreamComponents>) getStoredValue();
+    public void setReusableComponents(Analyzer analyzer, String fieldName, TokenStreamComponents components) {
+      Map<String, TokenStreamComponents> componentsPerField = (Map<String, TokenStreamComponents>) getStoredValue(analyzer);
       if (componentsPerField == null) {
         componentsPerField = new HashMap<String, TokenStreamComponents>();
-        setStoredValue(componentsPerField);
+        setStoredValue(analyzer, componentsPerField);
       }
       componentsPerField.put(fieldName, components);
     }
-  }
+  };
 
 }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java Fri Aug 30 15:06:42 2013
@@ -34,10 +34,27 @@ public abstract class AnalyzerWrapper ex
 
   /**
    * Creates a new AnalyzerWrapper.  Since the {@link Analyzer.ReuseStrategy} of
-   * the wrapped Analyzers are unknown, {@link Analyzer.PerFieldReuseStrategy} is assumed
+   * the wrapped Analyzers are unknown, {@link #PER_FIELD_REUSE_STRATEGY} is assumed.
+   * @deprecated Use {@link #AnalyzerWrapper(Analyzer.ReuseStrategy)}
+   * and specify a valid {@link Analyzer.ReuseStrategy}, probably retrieved from the
+   * wrapped analyzer using {@link #getReuseStrategy()}.
    */
+  @Deprecated
   protected AnalyzerWrapper() {
-    super(new PerFieldReuseStrategy());
+    this(PER_FIELD_REUSE_STRATEGY);
+  }
+
+  /**
+   * Creates a new AnalyzerWrapper with the given reuse strategy.
+   * <p>If you want to wrap a single delegate Analyzer you can probably
+   * reuse its strategy when instantiating this subclass:
+   * {@code super(delegate.getReuseStrategy());}.
+   * <p>If you choose different analyzers per field, use
+   * {@link #PER_FIELD_REUSE_STRATEGY}.
+   * @see #getReuseStrategy()
+   */
+  protected AnalyzerWrapper(ReuseStrategy reuseStrategy) {
+    super(reuseStrategy);
   }
 
   /**

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/TokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/TokenStream.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/TokenStream.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/analysis/TokenStream.java Fri Aug 30 15:06:42 2013
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.Closeable;
 import java.lang.reflect.Modifier;
 
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexWriter;
@@ -159,11 +160,18 @@ public abstract class TokenStream extend
    * setting the final offset of a stream. The final offset of a stream might
    * differ from the offset of the last token eg in case one or more whitespaces
    * followed after the last token, but a WhitespaceTokenizer was used.
+   * <p>
+   * Additionally any skipped positions (such as those removed by a stopfilter)
+   * can be applied to the position increment, or any adjustment of other
+   * attributes where the end-of-stream value may be important.
    * 
    * @throws IOException If an I/O error occurs
    */
   public void end() throws IOException {
-    // do nothing by default
+    clearAttributes(); // LUCENE-3849: don't consume dirty atts
+    if (hasAttribute(PositionIncrementAttribute.class)) {
+      getAttribute(PositionIncrementAttribute.class).setPositionIncrement(0);
+    }
   }
 
   /**

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/Codec.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/Codec.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/Codec.java Fri Aug 30 15:06:42 2013
@@ -119,7 +119,7 @@ public abstract class Codec implements N
     loader.reload(classloader);
   }
   
-  private static Codec defaultCodec = Codec.forName("Lucene42");
+  private static Codec defaultCodec = Codec.forName("Lucene45");
   
   /** expert: returns the default codec used for newly created
    *  {@link IndexWriterConfig}s.

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Fri Aug 30 15:06:42 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/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java Fri Aug 30 15:06:42 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,67 @@ 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;
+    
+    /** Creates a {@link Bits} returning true if the document has a value */
+    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;
+    
+    /** Creates a {@link Bits} returning true if the document has a value */
+    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;
+    }
+  }
 }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java Fri Aug 30 15:06:42 2013
@@ -21,13 +21,13 @@ package org.apache.lucene.codecs;
  * A codec that forwards all its method calls to another codec.
  * <p>
  * Extend this class when you need to reuse the functionality of an existing
- * codec. For example, if you want to build a codec that redefines Lucene42's
+ * codec. For example, if you want to build a codec that redefines Lucene45's
  * {@link LiveDocsFormat}:
  * <pre class="prettyprint">
  *   public final class CustomCodec extends FilterCodec {
  *
  *     public CustomCodec() {
- *       super("CustomCodec", new Lucene42Codec());
+ *       super("CustomCodec", new Lucene45Codec());
  *     }
  *
  *     public LiveDocsFormat liveDocsFormat() {

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java Fri Aug 30 15:06:42 2013
@@ -27,7 +27,6 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 
 /**

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesFormat.java Fri Aug 30 15:06:42 2013
@@ -118,6 +118,11 @@ import org.apache.lucene.util.packed.Pac
  *        {@code BYTES_VAR_DEREF BYTES_VAR_DEREF} it doesn't apply deduplication of the document values.
  *    </li>
  * </ul>
+ * <p>
+ * Limitations:
+ * <ul>
+ *   <li> Binary doc values can be at most {@link #MAX_BINARY_FIELD_LENGTH} in length.
+ * </ul>
  * @deprecated Only for reading old 4.0 and 4.1 segments
  */
 @Deprecated
@@ -125,6 +130,9 @@ import org.apache.lucene.util.packed.Pac
 // for back compat only!
 public class Lucene40DocValuesFormat extends DocValuesFormat {
   
+  /** Maximum length for each binary doc values field. */
+  public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
+  
   /** Sole constructor. */
   public Lucene40DocValuesFormat() {
     super("Lucene40");

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java Fri Aug 30 15:06:42 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/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java Fri Aug 30 15:06:42 2013
@@ -234,7 +234,6 @@ public final class Lucene41PostingsReade
     } else {
       termState.singletonDocID = -1;
       termState.docStartFP += in.readVLong();
-      System.out.println(termState.docStartFP);
     }
     if (fieldHasPositions) {
       termState.posStartFP += in.readVLong();
@@ -252,7 +251,6 @@ public final class Lucene41PostingsReade
     } else {
       termState.skipOffset = -1;
     }
-    //System.out.println("PR: state=" + termState);
   }
     
   @Override

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java Fri Aug 30 15:06:42 2013
@@ -17,7 +17,10 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FilterCodec;
@@ -32,6 +35,7 @@ import org.apache.lucene.codecs.lucene40
 import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+import org.apache.lucene.index.SegmentWriteState;
 
 /**
  * Implements the Lucene 4.2 index format, with configurable per-field postings
@@ -42,10 +46,12 @@ import org.apache.lucene.codecs.perfield
  *
  * @see org.apache.lucene.codecs.lucene42 package documentation for file format details.
  * @lucene.experimental
+ * @deprecated Only for reading old 4.2 segments
  */
 // NOTE: if we make largish changes in a minor release, easier to just make Lucene43Codec or whatever
 // if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
 // (it writes a minor version, etc).
+@Deprecated
 public class Lucene42Codec extends Codec {
   private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
   private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
@@ -129,10 +135,15 @@ public class Lucene42Codec extends Codec
   private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
   private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene42");
 
-  private final NormsFormat normsFormat = new Lucene42NormsFormat();
+  private final NormsFormat normsFormat = new Lucene42NormsFormat() {
+    @Override
+    public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
+      throw new UnsupportedOperationException("this codec can only be used for reading");
+    }
+  };
 
   @Override
-  public final NormsFormat normsFormat() {
+  public NormsFormat normsFormat() {
     return normsFormat;
   }
 }

Modified: lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java?rev=1518989&r1=1518988&r2=1518989&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene3069/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java Fri Aug 30 15:06:42 2013
@@ -118,8 +118,19 @@ import org.apache.lucene.util.packed.Blo
  *   <p>SortedSet entries store the list of ordinals in their BinaryData as a
  *      sequences of increasing {@link DataOutput#writeVLong vLong}s, delta-encoded.</p>       
  * </ol>
+ * <p>
+ * Limitations:
+ * <ul>
+ *   <li> Binary doc values can be at most {@link #MAX_BINARY_FIELD_LENGTH} in length.
+ * </ul>
+ * @deprecated Only for reading old 4.2 segments
  */
-public final class Lucene42DocValuesFormat extends DocValuesFormat {
+@Deprecated
+public class Lucene42DocValuesFormat extends DocValuesFormat {
+
+  /** Maximum length for each binary doc values field. */
+  public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
+  
   final float acceptableOverheadRatio;
   
   /** 
@@ -145,8 +156,7 @@ public final class Lucene42DocValuesForm
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    // note: we choose DEFAULT here (its reasonably fast, and for small bpv has tiny waste)
-    return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+    throw new UnsupportedOperationException("this codec can only be used for reading");
   }
   
   @Override
@@ -154,8 +164,8 @@ public final class Lucene42DocValuesForm
     return new Lucene42DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
   }
   
-  private static final String DATA_CODEC = "Lucene42DocValuesData";
-  private static final String DATA_EXTENSION = "dvd";
-  private static final String METADATA_CODEC = "Lucene42DocValuesMetadata";
-  private static final String METADATA_EXTENSION = "dvm";
+  static final String DATA_CODEC = "Lucene42DocValuesData";
+  static final String DATA_EXTENSION = "dvd";
+  static final String METADATA_CODEC = "Lucene42DocValuesMetadata";
+  static final String METADATA_EXTENSION = "dvm";
 }