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/10/21 20:58:44 UTC

svn commit: r1534320 [10/39] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/...

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Mon Oct 21 18:58:24 2013
@@ -20,7 +20,6 @@ package org.apache.lucene.codecs.lucene4
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -361,10 +360,8 @@ public class Lucene40TermVectorsReader e
     }
 
     @Override
-    public Comparator<BytesRef> getComparator() {
-      // TODO: really indexer hardwires
-      // this...?  I guess codec could buffer and re-sort...
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
+    public boolean hasFreqs() {
+      return true;
     }
 
     @Override
@@ -569,11 +566,6 @@ public class Lucene40TermVectorsReader e
       docsAndPositionsEnum.reset(liveDocs, positions, startOffsets, endOffsets, payloadOffsets, payloadData);
       return docsAndPositionsEnum;
     }
-
-    @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
   }
 
   // NOTE: sort of a silly class, since you can get the
@@ -763,5 +755,10 @@ public class Lucene40TermVectorsReader e
     
     return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs);
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
 }
 

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.TermVectorsReader;
@@ -443,9 +442,4 @@ public final class Lucene40TermVectorsWr
     IOUtils.close(tvx, tvd, tvf);
     tvx = tvd = tvf = null;
   }
-
-  @Override
-  public Comparator<BytesRef> getComparator() {
-    return BytesRef.getUTF8SortedAsUnicodeComparator();
-  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java Mon Oct 21 18:58:24 2013
@@ -101,7 +101,7 @@ public class Lucene41Codec extends Codec
   }
   
   @Override
-  public final SegmentInfoFormat segmentInfoFormat() {
+  public SegmentInfoFormat segmentInfoFormat() {
     return infosFormat;
   }
   

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java Mon Oct 21 18:58:24 2013
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene4
 import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
 import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
 import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
+import static org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter.IntBlockTermState;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -34,7 +35,6 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.TermState;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.Directory;
@@ -59,6 +59,7 @@ public final class Lucene41PostingsReade
   private final IndexInput payIn;
 
   private final ForUtil forUtil;
+  private int version;
 
   // public static boolean DEBUG = false;
 
@@ -71,27 +72,21 @@ public final class Lucene41PostingsReade
     try {
       docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
                             ioContext);
-      CodecUtil.checkHeader(docIn,
+      version = CodecUtil.checkHeader(docIn,
                             Lucene41PostingsWriter.DOC_CODEC,
-                            Lucene41PostingsWriter.VERSION_CURRENT,
+                            Lucene41PostingsWriter.VERSION_START,
                             Lucene41PostingsWriter.VERSION_CURRENT);
       forUtil = new ForUtil(docIn);
 
       if (fieldInfos.hasProx()) {
         posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
                               ioContext);
-        CodecUtil.checkHeader(posIn,
-                              Lucene41PostingsWriter.POS_CODEC,
-                              Lucene41PostingsWriter.VERSION_CURRENT,
-                              Lucene41PostingsWriter.VERSION_CURRENT);
+        CodecUtil.checkHeader(posIn, Lucene41PostingsWriter.POS_CODEC, version, version);
 
         if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
           payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
                                 ioContext);
-          CodecUtil.checkHeader(payIn,
-                                Lucene41PostingsWriter.PAY_CODEC,
-                                Lucene41PostingsWriter.VERSION_CURRENT,
-                                Lucene41PostingsWriter.VERSION_CURRENT);
+          CodecUtil.checkHeader(payIn, Lucene41PostingsWriter.PAY_CODEC, version, version);
         }
       }
 
@@ -111,7 +106,7 @@ public final class Lucene41PostingsReade
     // Make sure we are talking to the matching postings writer
     CodecUtil.checkHeader(termsIn,
                           Lucene41PostingsWriter.TERMS_CODEC,
-                          Lucene41PostingsWriter.VERSION_CURRENT,
+                          Lucene41PostingsWriter.VERSION_START,
                           Lucene41PostingsWriter.VERSION_CURRENT);
     final int indexBlockSize = termsIn.readVInt();
     if (indexBlockSize != BLOCK_SIZE) {
@@ -141,54 +136,8 @@ public final class Lucene41PostingsReade
     }
   }
 
-  // Must keep final because we do non-standard clone
-  private final static class IntBlockTermState extends BlockTermState {
-    long docStartFP;
-    long posStartFP;
-    long payStartFP;
-    long skipOffset;
-    long lastPosBlockOffset;
-    // docid when there is a single pulsed posting, otherwise -1
-    // freq is always implicitly totalTermFreq in this case.
-    int singletonDocID;
-
-    // Only used by the "primary" TermState -- clones don't
-    // copy this (basically they are "transient"):
-    ByteArrayDataInput bytesReader;  // TODO: should this NOT be in the TermState...?
-    byte[] bytes;
-
-    @Override
-    public IntBlockTermState clone() {
-      IntBlockTermState other = new IntBlockTermState();
-      other.copyFrom(this);
-      return other;
-    }
-
-    @Override
-    public void copyFrom(TermState _other) {
-      super.copyFrom(_other);
-      IntBlockTermState other = (IntBlockTermState) _other;
-      docStartFP = other.docStartFP;
-      posStartFP = other.posStartFP;
-      payStartFP = other.payStartFP;
-      lastPosBlockOffset = other.lastPosBlockOffset;
-      skipOffset = other.skipOffset;
-      singletonDocID = other.singletonDocID;
-
-      // Do not copy bytes, bytesReader (else TermState is
-      // very heavy, ie drags around the entire block's
-      // byte[]).  On seek back, if next() is in fact used
-      // (rare!), they will be re-read from disk.
-    }
-
-    @Override
-    public String toString() {
-      return super.toString() + " docStartFP=" + docStartFP + " posStartFP=" + posStartFP + " payStartFP=" + payStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
-    }
-  }
-
   @Override
-  public IntBlockTermState newTermState() {
+  public BlockTermState newTermState() {
     return new IntBlockTermState();
   }
 
@@ -197,81 +146,69 @@ public final class Lucene41PostingsReade
     IOUtils.close(docIn, posIn, payIn);
   }
 
-  /* Reads but does not decode the byte[] blob holding
-     metadata for the current terms block */
-  @Override
-  public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
-    final IntBlockTermState termState = (IntBlockTermState) _termState;
-
-    final int numBytes = termsIn.readVInt();
-
-    if (termState.bytes == null) {
-      termState.bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
-      termState.bytesReader = new ByteArrayDataInput();
-    } else if (termState.bytes.length < numBytes) {
-      termState.bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
-    }
-
-    termsIn.readBytes(termState.bytes, 0, numBytes);
-    termState.bytesReader.reset(termState.bytes, 0, numBytes);
-  }
-
   @Override
-  public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
+  public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
     throws IOException {
     final IntBlockTermState termState = (IntBlockTermState) _termState;
-    final boolean isFirstTerm = termState.termBlockOrd == 0;
     final boolean fieldHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
     final boolean fieldHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
     final boolean fieldHasPayloads = fieldInfo.hasPayloads();
 
-    final DataInput in = termState.bytesReader;
-    if (isFirstTerm) {
-      if (termState.docFreq == 1) {
-        termState.singletonDocID = in.readVInt();
-        termState.docStartFP = 0;
-      } else {
-        termState.singletonDocID = -1;
-        termState.docStartFP = in.readVLong();
+    if (absolute) {
+      termState.docStartFP = 0;
+      termState.posStartFP = 0;
+      termState.payStartFP = 0;
+    }
+    if (version < Lucene41PostingsWriter.VERSION_META_ARRAY) {  // backward compatibility
+      _decodeTerm(in, fieldInfo, termState);
+      return;
+    }
+    termState.docStartFP += longs[0];
+    if (fieldHasPositions) {
+      termState.posStartFP += longs[1];
+      if (fieldHasOffsets || fieldHasPayloads) {
+        termState.payStartFP += longs[2];
       }
-      if (fieldHasPositions) {
-        termState.posStartFP = in.readVLong();
-        if (termState.totalTermFreq > BLOCK_SIZE) {
-          termState.lastPosBlockOffset = in.readVLong();
-        } else {
-          termState.lastPosBlockOffset = -1;
-        }
-        if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
-          termState.payStartFP = in.readVLong();
-        } else {
-          termState.payStartFP = -1;
-        }
+    }
+    if (termState.docFreq == 1) {
+      termState.singletonDocID = in.readVInt();
+    } else {
+      termState.singletonDocID = -1;
+    }
+    if (fieldHasPositions) {
+      if (termState.totalTermFreq > BLOCK_SIZE) {
+        termState.lastPosBlockOffset = in.readVLong();
+      } else {
+        termState.lastPosBlockOffset = -1;
       }
+    }
+    if (termState.docFreq > BLOCK_SIZE) {
+      termState.skipOffset = in.readVLong();
+    } else {
+      termState.skipOffset = -1;
+    }
+  }
+  private void _decodeTerm(DataInput in, FieldInfo fieldInfo, IntBlockTermState termState) throws IOException {
+    final boolean fieldHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+    final boolean fieldHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+    final boolean fieldHasPayloads = fieldInfo.hasPayloads();
+    if (termState.docFreq == 1) {
+      termState.singletonDocID = in.readVInt();
     } else {
-      if (termState.docFreq == 1) {
-        termState.singletonDocID = in.readVInt();
+      termState.singletonDocID = -1;
+      termState.docStartFP += in.readVLong();
+    }
+    if (fieldHasPositions) {
+      termState.posStartFP += in.readVLong();
+      if (termState.totalTermFreq > BLOCK_SIZE) {
+        termState.lastPosBlockOffset = in.readVLong();
       } else {
-        termState.singletonDocID = -1;
-        termState.docStartFP += in.readVLong();
+        termState.lastPosBlockOffset = -1;
       }
-      if (fieldHasPositions) {
-        termState.posStartFP += in.readVLong();
-        if (termState.totalTermFreq > BLOCK_SIZE) {
-          termState.lastPosBlockOffset = in.readVLong();
-        } else {
-          termState.lastPosBlockOffset = -1;
-        }
-        if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
-          long delta = in.readVLong();
-          if (termState.payStartFP == -1) {
-            termState.payStartFP = delta;
-          } else {
-            termState.payStartFP += delta;
-          }
-        }
+      if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
+        termState.payStartFP += in.readVLong();
       }
     }
-
     if (termState.docFreq > BLOCK_SIZE) {
       termState.skipOffset = in.readVLong();
     } else {
@@ -1604,4 +1541,10 @@ public final class Lucene41PostingsReade
       return docFreq;
     }
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
+
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java Mon Oct 21 18:58:24 2013
@@ -23,6 +23,7 @@ import org.apache.lucene.codecs.compress
 import org.apache.lucene.codecs.compressing.CompressingStoredFieldsIndexWriter;
 import org.apache.lucene.codecs.compressing.CompressionMode;
 import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
+import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -88,6 +89,11 @@ import org.apache.lucene.util.packed.Pac
  * <li>If documents are larger than 16KB then chunks will likely contain only
  * one document. However, documents can never spread across several chunks (all
  * fields of a single document are in the same chunk).</li>
+ * <li>When at least one document in a chunk is large enough so that the chunk
+ * is larger than 32KB, the chunk will actually be compressed in several LZ4
+ * blocks of 16KB. This allows {@link StoredFieldVisitor}s which are only
+ * interested in the first fields of a document to not have to decompress 10MB
+ * of data if the document is 10MB, but only 16KB.</li>
  * <li>Given that the original lengths are written in the metadata of the chunk,
  * the decompressor can leverage this information to stop decoding as soon as
  * enough data has been decompressed.</li>

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java Mon Oct 21 18:58:24 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();
@@ -89,12 +95,12 @@ public class Lucene42Codec extends Codec
   }
   
   @Override
-  public final FieldInfosFormat fieldInfosFormat() {
+  public FieldInfosFormat fieldInfosFormat() {
     return fieldInfosFormat;
   }
   
   @Override
-  public final SegmentInfoFormat segmentInfoFormat() {
+  public SegmentInfoFormat segmentInfoFormat() {
     return infosFormat;
   }
   
@@ -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/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java Mon Oct 21 18:58:24 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";
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Mon Oct 21 18:58:24 2013
@@ -17,13 +17,7 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.DELTA_COMPRESSED;
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.GCD_COMPRESSED;
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.TABLE_COMPRESSED;
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.UNCOMPRESSED;
-
 import java.io.IOException;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -48,6 +42,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
 import org.apache.lucene.util.fst.FST;
@@ -78,6 +73,22 @@ class Lucene42DocValuesProducer extends 
       new HashMap<Integer,FST<Long>>();
   
   private final int maxDoc;
+  
+  
+  static final byte NUMBER = 0;
+  static final byte BYTES = 1;
+  static final byte FST = 2;
+
+  static final int BLOCK_SIZE = 4096;
+  
+  static final byte DELTA_COMPRESSED = 0;
+  static final byte TABLE_COMPRESSED = 1;
+  static final byte UNCOMPRESSED = 2;
+  static final byte GCD_COMPRESSED = 3;
+  
+  static final int VERSION_START = 0;
+  static final int VERSION_GCD_COMPRESSION = 1;
+  static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
     
   Lucene42DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     maxDoc = state.segmentInfo.getDocCount();
@@ -88,8 +99,8 @@ class Lucene42DocValuesProducer extends 
     final int version;
     try {
       version = CodecUtil.checkHeader(in, metaCodec, 
-                                      Lucene42DocValuesConsumer.VERSION_START,
-                                      Lucene42DocValuesConsumer.VERSION_CURRENT);
+                                      VERSION_START,
+                                      VERSION_CURRENT);
       numerics = new HashMap<Integer,NumericEntry>();
       binaries = new HashMap<Integer,BinaryEntry>();
       fsts = new HashMap<Integer,FSTEntry>();
@@ -109,8 +120,8 @@ class Lucene42DocValuesProducer extends 
       String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.openInput(dataName, state.context);
       final int version2 = CodecUtil.checkHeader(data, dataCodec, 
-                                                 Lucene42DocValuesConsumer.VERSION_START,
-                                                 Lucene42DocValuesConsumer.VERSION_CURRENT);
+                                                 VERSION_START,
+                                                 VERSION_CURRENT);
       if (version != version2) {
         throw new CorruptIndexException("Format versions mismatch");
       }
@@ -127,7 +138,7 @@ class Lucene42DocValuesProducer extends 
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
       int fieldType = meta.readByte();
-      if (fieldType == Lucene42DocValuesConsumer.NUMBER) {
+      if (fieldType == NUMBER) {
         NumericEntry entry = new NumericEntry();
         entry.offset = meta.readLong();
         entry.format = meta.readByte();
@@ -140,11 +151,11 @@ class Lucene42DocValuesProducer extends 
           default:
                throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
         }
-        if (entry.format != Lucene42DocValuesConsumer.UNCOMPRESSED) {
+        if (entry.format != UNCOMPRESSED) {
           entry.packedIntsVersion = meta.readVInt();
         }
         numerics.put(fieldNumber, entry);
-      } else if (fieldType == Lucene42DocValuesConsumer.BYTES) {
+      } else if (fieldType == BYTES) {
         BinaryEntry entry = new BinaryEntry();
         entry.offset = meta.readLong();
         entry.numBytes = meta.readLong();
@@ -155,7 +166,7 @@ class Lucene42DocValuesProducer extends 
           entry.blockSize = meta.readVInt();
         }
         binaries.put(fieldNumber, entry);
-      } else if (fieldType == Lucene42DocValuesConsumer.FST) {
+      } else if (fieldType == FST) {
         FSTEntry entry = new FSTEntry();
         entry.offset = meta.readLong();
         entry.numOrds = meta.readVLong();
@@ -177,6 +188,11 @@ class Lucene42DocValuesProducer extends 
     return instance;
   }
   
+  @Override
+  public long ramBytesUsed() {
+    return RamUsageEstimator.sizeOf(this);
+  }
+  
   private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
     NumericEntry entry = numerics.get(field.number);
     data.seek(entry.offset);
@@ -429,6 +445,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 {
@@ -485,11 +510,6 @@ class Lucene42DocValuesProducer extends 
     }
 
     @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
-
-    @Override
     public SeekStatus seekCeil(BytesRef text) throws IOException {
       if (in.seekCeil(text) == null) {
         return SeekStatus.END;

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java Mon Oct 21 18:58:24 2013
@@ -83,10 +83,11 @@ import org.apache.lucene.store.DataOutpu
  * </ul>
  *
  * @lucene.experimental
+ * @deprecated Only for reading old 4.2-4.5 segments
  */
-public final class Lucene42FieldInfosFormat extends FieldInfosFormat {
+@Deprecated
+public class Lucene42FieldInfosFormat extends FieldInfosFormat {
   private final FieldInfosReader reader = new Lucene42FieldInfosReader();
-  private final FieldInfosWriter writer = new Lucene42FieldInfosWriter();
   
   /** Sole constructor. */
   public Lucene42FieldInfosFormat() {
@@ -99,7 +100,7 @@ public final class Lucene42FieldInfosFor
 
   @Override
   public FieldInfosWriter getFieldInfosWriter() throws IOException {
-    return writer;
+    throw new UnsupportedOperationException("this codec can only be used for reading");
   }
   
   /** Extension of field infos */

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java Mon Oct 21 18:58:24 2013
@@ -38,8 +38,10 @@ import org.apache.lucene.util.IOUtils;
  * Lucene 4.2 FieldInfos reader.
  * 
  * @lucene.experimental
+ * @deprecated Only for reading old 4.2-4.5 segments
  * @see Lucene42FieldInfosFormat
  */
+@Deprecated
 final class Lucene42FieldInfosReader extends FieldInfosReader {
 
   /** Sole constructor. */
@@ -47,7 +49,7 @@ final class Lucene42FieldInfosReader ext
   }
 
   @Override
-  public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
+  public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext iocontext) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene42FieldInfosFormat.EXTENSION);
     IndexInput input = directory.openInput(fileName, iocontext);
     

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

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

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

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java Mon Oct 21 18:58:24 2013
@@ -22,13 +22,13 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.Map;
-import java.util.ServiceLoader; // javadocs
+import java.util.ServiceLoader;
 import java.util.TreeMap;
 
-import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.NumericDocValues;
@@ -36,8 +36,10 @@ 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;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Enables per field docvalues support.
@@ -74,11 +76,10 @@ public abstract class PerFieldDocValuesF
   }
 
   @Override
-  public final DocValuesConsumer fieldsConsumer(SegmentWriteState state)
-      throws IOException {
+  public final DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     return new FieldsWriter(state);
   }
-  
+
   static class ConsumerAndSuffix implements Closeable {
     DocValuesConsumer consumer;
     int suffix;
@@ -95,7 +96,7 @@ public abstract class PerFieldDocValuesF
     private final Map<String,Integer> suffixes = new HashMap<String,Integer>();
     
     private final SegmentWriteState segmentWriteState;
-
+    
     public FieldsWriter(SegmentWriteState state) {
       segmentWriteState = state;
     }
@@ -121,32 +122,53 @@ public abstract class PerFieldDocValuesF
     }
 
     private DocValuesConsumer getInstance(FieldInfo field) throws IOException {
-      final DocValuesFormat format = getDocValuesFormatForField(field.name);
+      DocValuesFormat format = null;
+      if (field.getDocValuesGen() != -1) {
+        final String formatName = field.getAttribute(PER_FIELD_FORMAT_KEY);
+        // this means the field never existed in that segment, yet is applied updates
+        if (formatName != null) {
+          format = DocValuesFormat.forName(formatName);
+        }
+      }
+      if (format == null) {
+        format = getDocValuesFormatForField(field.name);
+      }
       if (format == null) {
         throw new IllegalStateException("invalid null DocValuesFormat for field=\"" + field.name + "\"");
       }
       final String formatName = format.getName();
       
       String previousValue = field.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
-      assert previousValue == null: "formatName=" + formatName + " prevValue=" + previousValue;
+      assert field.getDocValuesGen() != -1 || previousValue == null: "formatName=" + formatName + " prevValue=" + previousValue;
       
-      Integer suffix;
+      Integer suffix = null;
       
       ConsumerAndSuffix consumer = formats.get(format);
       if (consumer == null) {
         // First time we are seeing this format; create a new instance
+
+        if (field.getDocValuesGen() != -1) {
+          final String suffixAtt = field.getAttribute(PER_FIELD_SUFFIX_KEY);
+          // even when dvGen is != -1, it can still be a new field, that never
+          // existed in the segment, and therefore doesn't have the recorded
+          // attributes yet.
+          if (suffixAtt != null) {
+            suffix = Integer.valueOf(suffixAtt);
+          }
+        }
         
-        // bump the suffix
-        suffix = suffixes.get(formatName);
         if (suffix == null) {
-          suffix = 0;
-        } else {
-          suffix = suffix + 1;
+          // bump the suffix
+          suffix = suffixes.get(formatName);
+          if (suffix == null) {
+            suffix = 0;
+          } else {
+            suffix = suffix + 1;
+          }
         }
         suffixes.put(formatName, suffix);
         
-        final String segmentSuffix = getFullSegmentSuffix(field.name,
-                                                          segmentWriteState.segmentSuffix,
+        final String segmentSuffix = getFullSegmentSuffix(segmentWriteState.segmentSuffix,
                                                           getSuffix(formatName, Integer.toString(suffix)));
         consumer = new ConsumerAndSuffix();
         consumer.consumer = format.fieldsConsumer(new SegmentWriteState(segmentWriteState, segmentSuffix));
@@ -159,10 +181,10 @@ public abstract class PerFieldDocValuesF
       }
       
       previousValue = field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix));
-      assert previousValue == null;
+      assert field.getDocValuesGen() != -1 || previousValue == null : "suffix=" + Integer.toString(suffix) + " prevValue=" + previousValue;
 
       // TODO: we should only provide the "slice" of FIS
-      // that this PF actually sees ...
+      // that this DVF actually sees ...
       return consumer.consumer;
     }
 
@@ -177,14 +199,11 @@ public abstract class PerFieldDocValuesF
     return formatName + "_" + suffix;
   }
 
-  static String getFullSegmentSuffix(String fieldName, String outerSegmentSuffix, String segmentSuffix) {
+  static String getFullSegmentSuffix(String outerSegmentSuffix, String segmentSuffix) {
     if (outerSegmentSuffix.length() == 0) {
       return segmentSuffix;
     } else {
-      // TODO: support embedding; I think it should work but
-      // we need a test confirm to confirm
-      // return outerSegmentSuffix + "_" + segmentSuffix;
-      throw new IllegalStateException("cannot embed PerFieldPostingsFormat inside itself (field \"" + fieldName + "\" returned PerFieldPostingsFormat)");
+      return outerSegmentSuffix + "_" + segmentSuffix;
     }
   }
 
@@ -208,7 +227,7 @@ public abstract class PerFieldDocValuesF
               final String suffix = fi.getAttribute(PER_FIELD_SUFFIX_KEY);
               assert suffix != null;
               DocValuesFormat format = DocValuesFormat.forName(formatName);
-              String segmentSuffix = getSuffix(formatName, suffix);
+              String segmentSuffix = getFullSegmentSuffix(readState.segmentSuffix, getSuffix(formatName, suffix));
               if (!formats.containsKey(segmentSuffix)) {
                 formats.put(segmentSuffix, format.fieldsProducer(new SegmentReadState(readState, segmentSuffix)));
               }
@@ -265,6 +284,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 {
@@ -275,6 +300,16 @@ public abstract class PerFieldDocValuesF
     public DocValuesProducer clone() {
       return new FieldsReader(this);
     }
+
+    @Override
+    public long ramBytesUsed() {
+      long size = 0;
+      for (Map.Entry<String,DocValuesProducer> entry : formats.entrySet()) {
+        size += (entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR) + 
+            entry.getValue().ramBytesUsed();
+      }
+      return size;
+    }
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Mon Oct 21 18:58:24 2013
@@ -17,24 +17,28 @@ package org.apache.lucene.codecs.perfiel
  * limitations under the License.
  */
 
-import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.ServiceLoader; // javadocs
+import java.util.Set;
 import java.util.TreeMap;
+import java.util.TreeSet;
 
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.TermsConsumer;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import static org.apache.lucene.index.FilterAtomicReader.FilterFields;
 
 /**
  * Enables per field postings support.
@@ -64,96 +68,22 @@ public abstract class PerFieldPostingsFo
    *  segment suffix name for each field. */
   public static final String PER_FIELD_SUFFIX_KEY = PerFieldPostingsFormat.class.getSimpleName() + ".suffix";
 
-  
   /** Sole constructor. */
   public PerFieldPostingsFormat() {
     super(PER_FIELD_NAME);
   }
 
-  @Override
-  public final FieldsConsumer fieldsConsumer(SegmentWriteState state)
-      throws IOException {
-    return new FieldsWriter(state);
-  }
-  
-  static class FieldsConsumerAndSuffix implements Closeable {
-    FieldsConsumer consumer;
+  /** Group of fields written by one PostingsFormat */
+  static class FieldsGroup {
+    final Set<String> fields = new TreeSet<String>();
     int suffix;
-    
-    @Override
-    public void close() throws IOException {
-      consumer.close();
-    }
-  }
-    
-  private class FieldsWriter extends FieldsConsumer {
 
-    private final Map<PostingsFormat,FieldsConsumerAndSuffix> formats = new HashMap<PostingsFormat,FieldsConsumerAndSuffix>();
-    private final Map<String,Integer> suffixes = new HashMap<String,Integer>();
-    
-    private final SegmentWriteState segmentWriteState;
-
-    public FieldsWriter(SegmentWriteState state) {
-      segmentWriteState = state;
-    }
+    /** Custom SegmentWriteState for this group of fields,
+     *  with the segmentSuffix uniqueified for this
+     *  PostingsFormat */
+    SegmentWriteState state;
+  };
 
-    @Override
-    public TermsConsumer addField(FieldInfo field) throws IOException {
-      final PostingsFormat format = getPostingsFormatForField(field.name);
-      if (format == null) {
-        throw new IllegalStateException("invalid null PostingsFormat for field=\"" + field.name + "\"");
-      }
-      final String formatName = format.getName();
-      
-      String previousValue = field.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
-      assert previousValue == null;
-      
-      Integer suffix;
-      
-      FieldsConsumerAndSuffix consumer = formats.get(format);
-      if (consumer == null) {
-        // First time we are seeing this format; create a new instance
-        
-        // bump the suffix
-        suffix = suffixes.get(formatName);
-        if (suffix == null) {
-          suffix = 0;
-        } else {
-          suffix = suffix + 1;
-        }
-        suffixes.put(formatName, suffix);
-        
-        final String segmentSuffix = getFullSegmentSuffix(field.name,
-                                                          segmentWriteState.segmentSuffix,
-                                                          getSuffix(formatName, Integer.toString(suffix)));
-        consumer = new FieldsConsumerAndSuffix();
-        consumer.consumer = format.fieldsConsumer(new SegmentWriteState(segmentWriteState, segmentSuffix));
-        consumer.suffix = suffix;
-        formats.put(format, consumer);
-      } else {
-        // we've already seen this format, so just grab its suffix
-        assert suffixes.containsKey(formatName);
-        suffix = consumer.suffix;
-      }
-      
-      previousValue = field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix));
-      assert previousValue == null;
-
-      // TODO: we should only provide the "slice" of FIS
-      // that this PF actually sees ... then stuff like
-      // .hasProx could work correctly?
-      // NOTE: .hasProx is already broken in the same way for the non-perfield case,
-      // if there is a fieldinfo with prox that has no postings, you get a 0 byte file.
-      return consumer.consumer.addField(field);
-    }
-
-    @Override
-    public void close() throws IOException {
-      // Close all subs
-      IOUtils.close(formats.values());
-    }
-  }
-  
   static String getSuffix(String formatName, String suffix) {
     return formatName + "_" + suffix;
   }
@@ -168,6 +98,87 @@ public abstract class PerFieldPostingsFo
       throw new IllegalStateException("cannot embed PerFieldPostingsFormat inside itself (field \"" + fieldName + "\" returned PerFieldPostingsFormat)");
     }
   }
+  
+  private class FieldsWriter extends FieldsConsumer {
+    final SegmentWriteState writeState;
+
+    public FieldsWriter(SegmentWriteState writeState) {
+      this.writeState = writeState;
+    }
+
+    @Override
+    public void write(Fields fields) throws IOException {
+
+      // Maps a PostingsFormat instance to the suffix it
+      // should use
+      Map<PostingsFormat,FieldsGroup> formatToGroups = new HashMap<PostingsFormat,FieldsGroup>();
+
+      // Holds last suffix of each PostingFormat name
+      Map<String,Integer> suffixes = new HashMap<String,Integer>();
+
+      // First pass: assign field -> PostingsFormat
+      for(String field : fields) {
+        FieldInfo fieldInfo = writeState.fieldInfos.fieldInfo(field);
+
+        final PostingsFormat format = getPostingsFormatForField(field);
+  
+        if (format == null) {
+          throw new IllegalStateException("invalid null PostingsFormat for field=\"" + field + "\"");
+        }
+        String formatName = format.getName();
+      
+        FieldsGroup group = formatToGroups.get(format);
+        if (group == null) {
+          // First time we are seeing this format; create a
+          // new instance
+
+          // bump the suffix
+          Integer suffix = suffixes.get(formatName);
+          if (suffix == null) {
+            suffix = 0;
+          } else {
+            suffix = suffix + 1;
+          }
+          suffixes.put(formatName, suffix);
+
+          String segmentSuffix = getFullSegmentSuffix(field,
+                                                      writeState.segmentSuffix,
+                                                      getSuffix(formatName, Integer.toString(suffix)));
+          group = new FieldsGroup();
+          group.state = new SegmentWriteState(writeState, segmentSuffix);
+          group.suffix = suffix;
+          formatToGroups.put(format, group);
+        } else {
+          // we've already seen this format, so just grab its suffix
+          assert suffixes.containsKey(formatName);
+        }
+
+        group.fields.add(field);
+
+        String previousValue = fieldInfo.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
+        assert previousValue == null;
+
+        previousValue = fieldInfo.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(group.suffix));
+        assert previousValue == null;
+      }
+
+      // Second pass: write postings
+      for(Map.Entry<PostingsFormat,FieldsGroup> ent : formatToGroups.entrySet()) {
+        PostingsFormat format = ent.getKey();
+        final FieldsGroup group = ent.getValue();
+
+        // Exposes only the fields from this group:
+        Fields maskedFields = new FilterFields(fields) {
+            @Override
+            public Iterator<String> iterator() {
+              return group.fields.iterator();
+            }
+          };
+
+        format.fieldsConsumer(group.state).write(maskedFields);
+      }
+    }
+  }
 
   private class FieldsReader extends FieldsProducer {
 
@@ -225,6 +236,22 @@ public abstract class PerFieldPostingsFo
     public void close() throws IOException {
       IOUtils.close(formats.values());
     }
+
+    @Override
+    public long ramBytesUsed() {
+      long sizeInBytes = 0;
+      for(Map.Entry<String,FieldsProducer> entry: formats.entrySet()) {
+        sizeInBytes += entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR;
+        sizeInBytes += entry.getValue().ramBytesUsed();
+      }
+      return sizeInBytes;
+    }
+  }
+
+  @Override
+  public final FieldsConsumer fieldsConsumer(SegmentWriteState state)
+      throws IOException {
+    return new FieldsWriter(state);
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Mon Oct 21 18:58:24 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/lucene4956/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
@@ -65,7 +64,6 @@ class AutomatonTermsEnum extends Filtere
   // of terms where we should simply do sequential reads instead.
   private boolean linear = false;
   private final BytesRef linearUpperBound = new BytesRef(10);
-  private final Comparator<BytesRef> termComp;
 
   /**
    * Construct an enumerator based upon an automaton, enumerating the specified
@@ -85,8 +83,6 @@ class AutomatonTermsEnum extends Filtere
 
     // used for path tracking, where each bit is a numbered state.
     visited = new long[runAutomaton.getSize()];
-
-    termComp = getComparator();
   }
   
   /**
@@ -99,10 +95,10 @@ class AutomatonTermsEnum extends Filtere
       if (runAutomaton.run(term.bytes, term.offset, term.length))
         return linear ? AcceptStatus.YES : AcceptStatus.YES_AND_SEEK;
       else
-        return (linear && termComp.compare(term, linearUpperBound) < 0) ? 
+        return (linear && term.compareTo(linearUpperBound) < 0) ? 
             AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
     } else {
-      return (linear && termComp.compare(term, linearUpperBound) < 0) ? 
+      return (linear && term.compareTo(linearUpperBound) < 0) ? 
           AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
     }
   }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java Mon Oct 21 18:58:24 2013
@@ -30,17 +30,12 @@ public abstract class BinaryDocValues {
 
   /** Lookup the value for document. */
   public abstract void get(int docID, BytesRef result);
-
-  /**
-   * Indicates the value was missing for the document.
-   */
-  public static final byte[] MISSING = new byte[0];
   
-  /** An empty BinaryDocValues which returns {@link #MISSING} for every document */
+  /** An empty BinaryDocValues which returns {@link BytesRef#EMPTY_BYTES} for every document */
   public static final BinaryDocValues EMPTY = new BinaryDocValues() {
     @Override
     public void get(int docID, BytesRef result) {
-      result.bytes = MISSING;
+      result.bytes = BytesRef.EMPTY_BYTES;
       result.offset = 0;
       result.length = 0;
     }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java Mon Oct 21 18:58:24 2013
@@ -22,29 +22,49 @@ import java.util.Iterator;
 import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
-import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
 import org.apache.lucene.util.packed.PackedInts;
 
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
-
-
 /** Buffers up pending byte[] per doc, then flushes when
  *  segment flushes. */
 class BinaryDocValuesWriter extends DocValuesWriter {
 
-  private final ByteBlockPool pool;
+  /** Maximum length for a binary field; we set this to "a
+   *  bit" below Integer.MAX_VALUE because the exact max
+   *  allowed byte[] is JVM dependent, so we want to avoid
+   *  a case where a large value worked in one JVM but
+   *  failed later at search time with a different JVM.  */
+  private static final int MAX_LENGTH = Integer.MAX_VALUE-256;
+
+  // 32 KB block sizes for PagedBytes storage:
+  private final static int BLOCK_BITS = 15;
+
+  private final PagedBytes bytes;
+  private final DataOutput bytesOut;
+
+  private final Counter iwBytesUsed;
   private final AppendingDeltaPackedLongBuffer lengths;
+  private final OpenBitSet docsWithField;
   private final FieldInfo fieldInfo;
-  private int addedValues = 0;
+  private int addedValues;
+  private long bytesUsed;
 
   public BinaryDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
     this.fieldInfo = fieldInfo;
-    this.pool = new ByteBlockPool(new DirectTrackingAllocator(iwBytesUsed));
+    this.bytes = new PagedBytes(BLOCK_BITS);
+    this.bytesOut = bytes.getDataOutput();
     this.lengths = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
+    this.iwBytesUsed = iwBytesUsed;
+    this.docsWithField = new OpenBitSet();
+    this.bytesUsed = docsWithFieldBytesUsed();
+    iwBytesUsed.addAndGet(bytesUsed);
   }
 
   public void addValue(int docID, BytesRef value) {
@@ -54,10 +74,10 @@ class BinaryDocValuesWriter extends DocV
     if (value == null) {
       throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\": null value not allowed");
     }
-    if (value.length > (BYTE_BLOCK_SIZE - 2)) {
-      throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" is too large, must be <= " + (BYTE_BLOCK_SIZE - 2));
+    if (value.length > MAX_LENGTH) {
+      throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" is too large, must be <= " + MAX_LENGTH);
     }
-    
+
     // Fill in any holes:
     while(addedValues < docID) {
       addedValues++;
@@ -65,7 +85,25 @@ class BinaryDocValuesWriter extends DocV
     }
     addedValues++;
     lengths.add(value.length);
-    pool.append(value);
+    try {
+      bytesOut.writeBytes(value.bytes, value.offset, value.length);
+    } catch (IOException ioe) {
+      // Should never happen!
+      throw new RuntimeException(ioe);
+    }
+    docsWithField.set(docID);
+    updateBytesUsed();
+  }
+  
+  private long docsWithFieldBytesUsed() {
+    // size of the long[] + some overhead
+    return RamUsageEstimator.sizeOf(docsWithField.getBits()) + 64;
+  }
+
+  private void updateBytesUsed() {
+    final long newBytesUsed = lengths.ramBytesUsed() + bytes.ramBytesUsed() + docsWithFieldBytesUsed();
+    iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
+    bytesUsed = newBytesUsed;
   }
 
   @Override
@@ -75,6 +113,7 @@ class BinaryDocValuesWriter extends DocV
   @Override
   public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
     final int maxDoc = state.segmentInfo.getDocCount();
+    bytes.freeze(false);
     dvConsumer.addBinaryField(fieldInfo,
                               new Iterable<BytesRef>() {
                                 @Override
@@ -92,10 +131,10 @@ class BinaryDocValuesWriter extends DocV
   private class BytesIterator implements Iterator<BytesRef> {
     final BytesRef value = new BytesRef();
     final AppendingDeltaPackedLongBuffer.Iterator lengthsIterator = lengths.iterator();
+    final DataInput bytesIterator = bytes.getDataInput();
     final int size = (int) lengths.size();
     final int maxDoc;
     int upto;
-    long byteOffset;
     
     BytesIterator(int maxDoc) {
       this.maxDoc = maxDoc;
@@ -111,19 +150,27 @@ 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;
+        try {
+          bytesIterator.readBytes(value.bytes, value.offset, value.length);
+        } catch (IOException ioe) {
+          // Should never happen!
+          throw new RuntimeException(ioe);
+        }
+        if (docsWithField.get(upto)) {
+          v = value;
+        } else {
+          v = null;
+        }
       } else {
-        // This is to handle last N documents not having
-        // this DV field in the end of the segment:
-        value.length = 0;
+        v = null;
       }
       upto++;
-      return value;
+      return v;
     }
 
     @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java Mon Oct 21 18:58:24 2013
@@ -19,25 +19,26 @@ package org.apache.lucene.index;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.RamUsageEstimator;
 
-/* Holds buffered deletes, by docID, term or query for a
+/* Holds buffered deletes and updates, by docID, term or query for a
  * single segment. This is used to hold buffered pending
- * deletes against the to-be-flushed segment.  Once the
- * deletes are pushed (on flush in DocumentsWriter), these
- * deletes are converted to a FrozenDeletes instance. */
+ * deletes and updates against the to-be-flushed segment.  Once the
+ * deletes and updates are pushed (on flush in DocumentsWriter), they
+ * are converted to a FrozenDeletes instance. */
 
 // NOTE: instances of this class are accessed either via a private
 // instance on DocumentWriterPerThread, or via sync'd code by
 // DocumentsWriterDeleteQueue
 
-class BufferedDeletes {
+class BufferedDeletes { // TODO (DVU_RENAME) BufferedUpdates?
 
   /* Rough logic: HashMap has an array[Entry] w/ varying
      load factor (say 2 * POINTER).  Entry is object w/ Term
@@ -63,11 +64,50 @@ class BufferedDeletes {
      undercount (say 24 bytes).  Integer is OBJ_HEADER + INT. */
   final static int BYTES_PER_DEL_QUERY = 5*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 2*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2*RamUsageEstimator.NUM_BYTES_INT + 24;
 
+  /* Rough logic: NumericUpdate calculates its actual size,
+   * including the update Term and DV field (String). The 
+   * per-field map holds a reference to the updated field, and
+   * therefore we only account for the object reference and 
+   * map space itself. This is incremented when we first see
+   * an updated field.
+   * 
+   * HashMap has an array[Entry] w/ varying load
+   * factor (say 2*POINTER). Entry is an object w/ String key, 
+   * LinkedHashMap val, int hash, Entry next (OBJ_HEADER + 3*POINTER + INT).
+   * 
+   * LinkedHashMap (val) is counted as OBJ_HEADER, array[Entry] ref + header, 4*INT, 1*FLOAT,
+   * Set (entrySet) (2*OBJ_HEADER + ARRAY_HEADER + 2*POINTER + 4*INT + FLOAT)
+   */
+  final static int BYTES_PER_NUMERIC_FIELD_ENTRY =
+      7*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 3*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 
+      RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + 5*RamUsageEstimator.NUM_BYTES_INT + RamUsageEstimator.NUM_BYTES_FLOAT;
+      
+  /* Rough logic: Incremented when we see another Term for an already updated
+   * field.
+   * LinkedHashMap has an array[Entry] w/ varying load factor 
+   * (say 2*POINTER). Entry is an object w/ Term key, NumericUpdate val, 
+   * int hash, Entry next, Entry before, Entry after (OBJ_HEADER + 5*POINTER + INT).
+   * 
+   * Term (key) is counted only as POINTER.
+   * NumericUpdate (val) counts its own size and isn't accounted for here.
+   */
+  final static int BYTES_PER_NUMERIC_UPDATE_ENTRY = 7*RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.NUM_BYTES_INT;
+  
   final AtomicInteger numTermDeletes = new AtomicInteger();
+  final AtomicInteger numNumericUpdates = new AtomicInteger();
   final Map<Term,Integer> terms = new HashMap<Term,Integer>();
   final Map<Query,Integer> queries = new HashMap<Query,Integer>();
   final List<Integer> docIDs = new ArrayList<Integer>();
 
+  // Map<dvField,Map<updateTerm,NumericUpdate>>
+  // For each field we keep an ordered list of NumericUpdates, key'd by the
+  // update Term. LinkedHashMap guarantees we will later traverse the map in
+  // insertion order (so that if two terms affect the same document, the last
+  // one that came in wins), and helps us detect faster if the same Term is
+  // used to update the same field multiple times (so we later traverse it
+  // only once).
+  final Map<String,LinkedHashMap<Term,NumericUpdate>> numericUpdates = new HashMap<String,LinkedHashMap<Term,NumericUpdate>>();
+
   public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE);
 
   final AtomicLong bytesUsed;
@@ -75,21 +115,17 @@ class BufferedDeletes {
   private final static boolean VERBOSE_DELETES = false;
 
   long gen;
+  
   public BufferedDeletes() {
-    this(new AtomicLong());
-  }
-
-  BufferedDeletes(AtomicLong bytesUsed) {
-    assert bytesUsed != null;
-    this.bytesUsed = bytesUsed;
+    this.bytesUsed = new AtomicLong();
   }
 
   @Override
   public String toString() {
     if (VERBOSE_DELETES) {
       return "gen=" + gen + " numTerms=" + numTermDeletes + ", terms=" + terms
-        + ", queries=" + queries + ", docIDs=" + docIDs + ", bytesUsed="
-        + bytesUsed;
+        + ", queries=" + queries + ", docIDs=" + docIDs + ", numericUpdates=" + numericUpdates
+        + ", bytesUsed=" + bytesUsed;
     } else {
       String s = "gen=" + gen;
       if (numTermDeletes.get() != 0) {
@@ -101,6 +137,9 @@ class BufferedDeletes {
       if (docIDs.size() != 0) {
         s += " " + docIDs.size() + " deleted docIDs";
       }
+      if (numNumericUpdates.get() != 0) {
+        s += " " + numNumericUpdates.get() + " numeric updates (unique count=" + numericUpdates.size() + ")";
+      }
       if (bytesUsed.get() != 0) {
         s += " bytesUsed=" + bytesUsed.get();
       }
@@ -145,20 +184,46 @@ class BufferedDeletes {
     }
   }
  
+  public void addNumericUpdate(NumericUpdate update, int docIDUpto) {
+    LinkedHashMap<Term,NumericUpdate> fieldUpdates = numericUpdates.get(update.field);
+    if (fieldUpdates == null) {
+      fieldUpdates = new LinkedHashMap<Term,NumericUpdate>();
+      numericUpdates.put(update.field, fieldUpdates);
+      bytesUsed.addAndGet(BYTES_PER_NUMERIC_FIELD_ENTRY);
+    }
+    final NumericUpdate current = fieldUpdates.get(update.term);
+    if (current != null && docIDUpto < current.docIDUpto) {
+      // Only record the new number if it's greater than or equal to the current
+      // one. This is important because if multiple threads are replacing the
+      // same doc at nearly the same time, it's possible that one thread that
+      // got a higher docID is scheduled before the other threads.
+      return;
+    }
+
+    update.docIDUpto = docIDUpto;
+    // since it's a LinkedHashMap, we must first remove the Term entry so that
+    // it's added last (we're interested in insertion-order).
+    if (current != null) {
+      fieldUpdates.remove(update.term);
+    }
+    fieldUpdates.put(update.term, update);
+    numNumericUpdates.incrementAndGet();
+    if (current == null) {
+      bytesUsed.addAndGet(BYTES_PER_NUMERIC_UPDATE_ENTRY + update.sizeInBytes());
+    }
+  }
+  
   void clear() {
     terms.clear();
     queries.clear();
     docIDs.clear();
+    numericUpdates.clear();
     numTermDeletes.set(0);
+    numNumericUpdates.set(0);
     bytesUsed.set(0);
   }
   
-  void clearDocIDs() {
-    bytesUsed.addAndGet(-docIDs.size()*BYTES_PER_DEL_DOCID);
-    docIDs.clear();
-  }
-  
   boolean any() {
-    return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0;
+    return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0 || numericUpdates.size() > 0;
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java Mon Oct 21 18:58:24 2013
@@ -18,10 +18,13 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.List;
 import java.util.ArrayList;
-import java.util.Comparator;
+import java.util.Arrays;
 import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -35,12 +38,12 @@ import org.apache.lucene.util.InfoStream
 
 /* Tracks the stream of {@link BufferedDeletes}.
  * When DocumentsWriterPerThread flushes, its buffered
- * deletes are appended to this stream.  We later
- * apply these deletes (resolve them to the actual
+ * deletes and updates are appended to this stream.  We later
+ * apply them (resolve them to the actual
  * docIDs, per segment) when a merge is started
  * (only to the to-be-merged segments).  We
  * also apply to all segments when NRT reader is pulled,
- * commit/close is called, or when too many deletes are
+ * commit/close is called, or when too many deletes or  updates are
  * buffered and must be flushed (by RAM usage or by count).
  *
  * Each packet is assigned a generation, and each flushed or
@@ -48,7 +51,7 @@ import org.apache.lucene.util.InfoStream
  * track which BufferedDeletes packets to apply to any given
  * segment. */
 
-class BufferedDeletesStream {
+class BufferedDeletesStream { // TODO (DVU_RENAME) BufferedUpdatesStream
 
   // TODO: maybe linked list?
   private final List<FrozenBufferedDeletes> deletes = new ArrayList<FrozenBufferedDeletes>();
@@ -114,6 +117,7 @@ class BufferedDeletesStream {
   }
 
   public static class ApplyDeletesResult {
+    
     // True if any actual deletes took place:
     public final boolean anyDeletes;
 
@@ -183,7 +187,7 @@ class BufferedDeletesStream {
       final long segGen = info.getBufferedDeletesGen();
 
       if (packet != null && segGen < packet.delGen()) {
-        //System.out.println("  coalesce");
+//        System.out.println("  coalesce");
         if (coalescedDeletes == null) {
           coalescedDeletes = new CoalescedDeletes();
         }
@@ -210,15 +214,21 @@ class BufferedDeletesStream {
         int delCount = 0;
         final boolean segAllDeletes;
         try {
+          Map<String,NumericFieldUpdates> fieldUpdates = null;
           if (coalescedDeletes != null) {
             //System.out.println("    del coalesced");
             delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader);
             delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader);
+            fieldUpdates = applyNumericDocValuesUpdates(coalescedDeletes.numericDVUpdates, rld, reader, fieldUpdates);
           }
           //System.out.println("    del exact");
           // Don't delete by Term here; DocumentsWriterPerThread
           // already did that on flush:
           delCount += applyQueryDeletes(packet.queriesIterable(), rld, reader);
+          fieldUpdates = applyNumericDocValuesUpdates(Arrays.asList(packet.updates), rld, reader, fieldUpdates);
+          if (!fieldUpdates.isEmpty()) {
+            rld.writeFieldUpdates(info.info.dir, fieldUpdates);
+          }
           final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
           assert fullDelCount <= rld.info.info.getDocCount();
           segAllDeletes = fullDelCount == rld.info.info.getDocCount();
@@ -265,10 +275,14 @@ class BufferedDeletesStream {
           try {
             delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader);
             delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader);
+            Map<String,NumericFieldUpdates> fieldUpdates = applyNumericDocValuesUpdates(coalescedDeletes.numericDVUpdates, rld, reader, null);
+            if (!fieldUpdates.isEmpty()) {
+              rld.writeFieldUpdates(info.info.dir, fieldUpdates);
+            }
             final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
             assert fullDelCount <= rld.info.info.getDocCount();
             segAllDeletes = fullDelCount == rld.info.info.getDocCount();
-          } finally {   
+          } finally {
             rld.release(reader);
             readerPool.release(rld);
           }
@@ -282,7 +296,7 @@ class BufferedDeletesStream {
           }
 
           if (infoStream.isEnabled("BD")) {
-            infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] newDelCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
+            infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + coalescedDeletes + "] newDelCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
           }
         }
         info.setBufferedDeletesGen(gen);
@@ -377,7 +391,7 @@ class BufferedDeletesStream {
         currentField = term.field();
         Terms terms = fields.terms(currentField);
         if (terms != null) {
-          termsEnum = terms.iterator(null);
+          termsEnum = terms.iterator(termsEnum);
         } else {
           termsEnum = null;
         }
@@ -402,15 +416,15 @@ class BufferedDeletesStream {
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
             }   
+            if (!any) {
+              rld.initWritableLiveDocs();
+              any = true;
+            }
             // NOTE: there is no limit check on the docID
             // when deleting by Term (unlike by Query)
             // because on flush we apply all Term deletes to
             // each segment.  So all Term deleting here is
             // against prior segments:
-            if (!any) {
-              rld.initWritableLiveDocs();
-              any = true;
-            }
             if (rld.delete(docID)) {
               delCount++;
             }
@@ -422,6 +436,87 @@ class BufferedDeletesStream {
     return delCount;
   }
 
+  // NumericDocValues Updates
+  // If otherFieldUpdates != null, we need to merge the updates into them
+  private synchronized Map<String,NumericFieldUpdates> applyNumericDocValuesUpdates(Iterable<NumericUpdate> updates, 
+      ReadersAndLiveDocs rld, SegmentReader reader, Map<String,NumericFieldUpdates> otherFieldUpdates) throws IOException {
+    Fields fields = reader.fields();
+    if (fields == null) {
+      // This reader has no postings
+      return Collections.emptyMap();
+    }
+
+    // TODO: we can process the updates per DV field, from last to first so that
+    // if multiple terms affect same document for the same field, we add an update
+    // only once (that of the last term). To do that, we can keep a bitset which
+    // marks which documents have already been updated. So e.g. if term T1
+    // updates doc 7, and then we process term T2 and it updates doc 7 as well,
+    // we don't apply the update since we know T1 came last and therefore wins
+    // the update.
+    // We can also use that bitset as 'liveDocs' to pass to TermEnum.docs(), so
+    // that these documents aren't even returned.
+    
+    String currentField = null;
+    TermsEnum termsEnum = null;
+    DocsEnum docs = null;
+    final Map<String,NumericFieldUpdates> result = otherFieldUpdates == null ? new HashMap<String,NumericFieldUpdates>() : otherFieldUpdates;
+    //System.out.println(Thread.currentThread().getName() + " numericDVUpdate reader=" + reader);
+    for (NumericUpdate update : updates) {
+      Term term = update.term;
+      int limit = update.docIDUpto;
+      
+      // TODO: we traverse the terms in update order (not term order) so that we
+      // apply the updates in the correct order, i.e. if two terms udpate the
+      // same document, the last one that came in wins, irrespective of the
+      // terms lexical order.
+      // we can apply the updates in terms order if we keep an updatesGen (and
+      // increment it with every update) and attach it to each NumericUpdate. Note
+      // that we cannot rely only on docIDUpto because an app may send two updates
+      // which will get same docIDUpto, yet will still need to respect the order
+      // those updates arrived.
+      
+      if (!term.field().equals(currentField)) {
+        // if we change the code to process updates in terms order, enable this assert
+//        assert currentField == null || currentField.compareTo(term.field()) < 0;
+        currentField = term.field();
+        Terms terms = fields.terms(currentField);
+        if (terms != null) {
+          termsEnum = terms.iterator(termsEnum);
+        } else {
+          termsEnum = null;
+          continue; // no terms in that field
+        }
+      }
+
+      if (termsEnum == null) {
+        continue;
+      }
+      // System.out.println("  term=" + term);
+
+      if (termsEnum.seekExact(term.bytes())) {
+        // we don't need term frequencies for this
+        DocsEnum docsEnum = termsEnum.docs(rld.getLiveDocs(), docs, DocsEnum.FLAG_NONE);
+      
+        //System.out.println("BDS: got docsEnum=" + docsEnum);
+
+        NumericFieldUpdates fieldUpdates = result.get(update.field);
+        if (fieldUpdates == null) {
+          fieldUpdates = new NumericFieldUpdates.PackedNumericFieldUpdates(reader.maxDoc());
+          result.put(update.field, fieldUpdates);
+        }
+        int doc;
+        while ((doc = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+          //System.out.println(Thread.currentThread().getName() + " numericDVUpdate term=" + term + " doc=" + docID);
+          if (doc >= limit) {
+            break; // no more docs that can be updated for this term
+          }
+          fieldUpdates.add(doc, update.value);
+        }
+      }
+    }
+    return result;
+  }
+
   public static class QueryAndLimit {
     public final Query query;
     public final int limit;