You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/10/28 10:19:29 UTC

svn commit: r1634823 [3/9] - in /lucene/dev/branches/lucene6005: ./ dev-tools/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/ lucene/analysis/common/src/java/org/apache/lucene/analy...

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java Tue Oct 28 09:19:21 2014
@@ -30,17 +30,16 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 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.SegmentReadState;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.fst.ByteSequenceOutputs;
+import org.apache.lucene.util.fst.Outputs;
 
 /** A block-based terms index and dictionary that assigns
  *  terms to variable length blocks according to how they
@@ -74,8 +73,31 @@ import org.apache.lucene.util.IOUtils;
 
 public final class BlockTreeTermsReader extends FieldsProducer {
 
+  static final Outputs<BytesRef> FST_OUTPUTS = ByteSequenceOutputs.getSingleton();
+  
+  static final BytesRef NO_OUTPUT = FST_OUTPUTS.getNoOutput();
+
+  static final int OUTPUT_FLAGS_NUM_BITS = 2;
+  static final int OUTPUT_FLAGS_MASK = 0x3;
+  static final int OUTPUT_FLAG_IS_FLOOR = 0x1;
+  static final int OUTPUT_FLAG_HAS_TERMS = 0x2;
+
+  /** Extension of terms file */
+  static final String TERMS_EXTENSION = "tim";
+  final static String TERMS_CODEC_NAME = "BlockTreeTermsDict";
+
+  /** Initial terms format. */
+  public static final int VERSION_START = 0;
+
+  /** Current terms format. */
+  public static final int VERSION_CURRENT = VERSION_START;
+
+  /** Extension of terms index file */
+  static final String TERMS_INDEX_EXTENSION = "tip";
+  final static String TERMS_INDEX_CODEC_NAME = "BlockTreeTermsIndex";
+
   // Open input to the main terms dict file (_X.tib)
-  final IndexInput in;
+  final IndexInput termsIn;
 
   //private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
 
@@ -96,105 +118,86 @@ public final class BlockTreeTermsReader 
   private final int version;
 
   /** Sole constructor. */
-  public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
-                              PostingsReaderBase postingsReader, IOContext ioContext,
-                              String segmentSuffix)
-    throws IOException {
-    
-    this.postingsReader = postingsReader;
-
-    this.segment = info.name;
-    in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION),
-                       ioContext);
-
+  public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state) throws IOException {
     boolean success = false;
     IndexInput indexIn = null;
-
+    
+    this.postingsReader = postingsReader;
+    this.segment = state.segmentInfo.name;
+    
+    String termsName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, TERMS_EXTENSION);
     try {
-      version = readHeader(in);
-      indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
-                                ioContext);
-      int indexVersion = readIndexHeader(indexIn);
-      if (indexVersion != version) {
-        throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion, indexIn);
-      }
+      termsIn = state.directory.openInput(termsName, state.context);
+      version = CodecUtil.checkIndexHeader(termsIn, TERMS_CODEC_NAME, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
       
-      // verify
-      if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
-        CodecUtil.checksumEntireFile(indexIn);
-      }
+      String indexName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, TERMS_INDEX_EXTENSION);
+      indexIn = state.directory.openInput(indexName, state.context);
+      CodecUtil.checkIndexHeader(indexIn, TERMS_INDEX_CODEC_NAME, version, version, state.segmentInfo.getId(), state.segmentSuffix);
+      CodecUtil.checksumEntireFile(indexIn);
 
       // Have PostingsReader init itself
-      postingsReader.init(in);
-      
+      postingsReader.init(termsIn, state);
       
       // NOTE: data file is too costly to verify checksum against all the bytes on open,
       // but for now we at least verify proper structure of the checksum footer: which looks
       // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
       // such as file truncation.
-      if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
-        CodecUtil.retrieveChecksum(in);
-      }
+      CodecUtil.retrieveChecksum(termsIn);
 
       // Read per-field details
-      seekDir(in, dirOffset);
+      seekDir(termsIn, dirOffset);
       seekDir(indexIn, indexDirOffset);
 
-      final int numFields = in.readVInt();
+      final int numFields = termsIn.readVInt();
       if (numFields < 0) {
-        throw new CorruptIndexException("invalid numFields: " + numFields, in);
+        throw new CorruptIndexException("invalid numFields: " + numFields, termsIn);
       }
 
-      for(int i=0;i<numFields;i++) {
-        final int field = in.readVInt();
-        final long numTerms = in.readVLong();
+      for (int i = 0; i < numFields; ++i) {
+        final int field = termsIn.readVInt();
+        final long numTerms = termsIn.readVLong();
         if (numTerms <= 0) {
-          throw new CorruptIndexException("Illegal numTerms for field number: " + field, in);
+          throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsIn);
         }
-        final int numBytes = in.readVInt();
+        final int numBytes = termsIn.readVInt();
         if (numBytes < 0) {
-          throw new CorruptIndexException("invalid rootCode for field number: " + field + ", numBytes=" + numBytes, in);
+          throw new CorruptIndexException("invalid rootCode for field number: " + field + ", numBytes=" + numBytes, termsIn);
         }
         final BytesRef rootCode = new BytesRef(new byte[numBytes]);
-        in.readBytes(rootCode.bytes, 0, numBytes);
+        termsIn.readBytes(rootCode.bytes, 0, numBytes);
         rootCode.length = numBytes;
-        final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+        final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
         if (fieldInfo == null) {
-          throw new CorruptIndexException("invalid field number: " + field, in);
+          throw new CorruptIndexException("invalid field number: " + field, termsIn);
         }
-        final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
-        final long sumDocFreq = in.readVLong();
-        final int docCount = in.readVInt();
-        final int longsSize = version >= BlockTreeTermsWriter.VERSION_META_ARRAY ? in.readVInt() : 0;
+        final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : termsIn.readVLong();
+        final long sumDocFreq = termsIn.readVLong();
+        final int docCount = termsIn.readVInt();
+        final int longsSize = termsIn.readVInt();
         if (longsSize < 0) {
-          throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, in);
+          throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsIn);
         }
-        BytesRef minTerm, maxTerm;
-        if (version >= BlockTreeTermsWriter.VERSION_MIN_MAX_TERMS) {
-          minTerm = readBytesRef(in);
-          maxTerm = readBytesRef(in);
-        } else {
-          minTerm = maxTerm = null;
-        }
-        if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs
-          throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount(), in);
+        BytesRef minTerm = readBytesRef(termsIn);
+        BytesRef maxTerm = readBytesRef(termsIn);
+        if (docCount < 0 || docCount > state.segmentInfo.getDocCount()) { // #docs with field must be <= #docs
+          throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.getDocCount(), termsIn);
         }
         if (sumDocFreq < docCount) {  // #postings must be >= #docs with field
-          throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, in);
+          throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, termsIn);
         }
         if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
-          throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq, in);
+          throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq, termsIn);
         }
         final long indexStartFP = indexIn.readVLong();
         FieldReader previous = fields.put(fieldInfo.name,       
                                           new FieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount,
                                                           indexStartFP, longsSize, indexIn, minTerm, maxTerm));
         if (previous != null) {
-          throw new CorruptIndexException("duplicate field: " + fieldInfo.name, in);
+          throw new CorruptIndexException("duplicate field: " + fieldInfo.name, termsIn);
         }
       }
+      
       indexIn.close();
-
       success = true;
     } finally {
       if (!success) {
@@ -212,38 +215,11 @@ public final class BlockTreeTermsReader 
     return bytes;
   }
 
-  /** Reads terms file header. */
-  private int readHeader(IndexInput input) throws IOException {
-    int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
-                          BlockTreeTermsWriter.VERSION_START,
-                          BlockTreeTermsWriter.VERSION_CURRENT);
-    if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
-      dirOffset = input.readLong();
-    }
-    return version;
-  }
-
-  /** Reads index file header. */
-  private int readIndexHeader(IndexInput input) throws IOException {
-    int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
-                          BlockTreeTermsWriter.VERSION_START,
-                          BlockTreeTermsWriter.VERSION_CURRENT);
-    if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
-      indexDirOffset = input.readLong(); 
-    }
-    return version;
-  }
-
   /** Seek {@code input} to the directory offset. */
   private void seekDir(IndexInput input, long dirOffset)
       throws IOException {
-    if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
-      input.seek(input.length() - CodecUtil.footerLength() - 8);
-      dirOffset = input.readLong();
-    } else if (version >= BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
-      input.seek(input.length() - 8);
-      dirOffset = input.readLong();
-    }
+    input.seek(input.length() - CodecUtil.footerLength() - 8);
+    dirOffset = input.readLong();
     input.seek(dirOffset);
   }
 
@@ -255,7 +231,7 @@ public final class BlockTreeTermsReader 
   @Override
   public void close() throws IOException {
     try {
-      IOUtils.close(in, postingsReader);
+      IOUtils.close(termsIn, postingsReader);
     } finally { 
       // Clear so refs to terms index is GCable even if
       // app hangs onto us:
@@ -313,14 +289,12 @@ public final class BlockTreeTermsReader 
   }
 
   @Override
-  public void checkIntegrity() throws IOException {
-    if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {      
-      // term dictionary
-      CodecUtil.checksumEntireFile(in);
+  public void checkIntegrity() throws IOException { 
+    // term dictionary
+    CodecUtil.checksumEntireFile(termsIn);
       
-      // postings
-      postingsReader.checkIntegrity();
-    }
+    // postings
+    postingsReader.checkIntegrity();
   }
 
   @Override

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java Tue Oct 28 09:19:21 2014
@@ -41,14 +41,12 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.fst.Outputs;
 import org.apache.lucene.util.fst.Util;
 import org.apache.lucene.util.packed.PackedInts;
 
@@ -192,10 +190,6 @@ import org.apache.lucene.util.packed.Pac
  */
 public final class BlockTreeTermsWriter extends FieldsConsumer {
 
-  static final Outputs<BytesRef> FST_OUTPUTS = ByteSequenceOutputs.getSingleton();
-
-  static final BytesRef NO_OUTPUT = FST_OUTPUTS.getNoOutput();
-
   /** Suggested default value for the {@code
    *  minItemsInBlock} parameter to {@link
    *  #BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */
@@ -209,38 +203,7 @@ public final class BlockTreeTermsWriter 
   // public final static boolean DEBUG = false;
   //private final static boolean SAVE_DOT_FILES = false;
 
-  static final int OUTPUT_FLAGS_NUM_BITS = 2;
-  static final int OUTPUT_FLAGS_MASK = 0x3;
-  static final int OUTPUT_FLAG_IS_FLOOR = 0x1;
-  static final int OUTPUT_FLAG_HAS_TERMS = 0x2;
-
-  /** Extension of terms file */
-  static final String TERMS_EXTENSION = "tim";
-  final static String TERMS_CODEC_NAME = "BLOCK_TREE_TERMS_DICT";
-
-  /** Initial terms format. */
-  public static final int VERSION_START = 0;
-  
-  /** Append-only */
-  public static final int VERSION_APPEND_ONLY = 1;
-
-  /** Meta data as array */
-  public static final int VERSION_META_ARRAY = 2;
-  
-  /** checksums */
-  public static final int VERSION_CHECKSUM = 3;
-
-  /** min/max term */
-  public static final int VERSION_MIN_MAX_TERMS = 4;
-
-  /** Current terms format. */
-  public static final int VERSION_CURRENT = VERSION_MIN_MAX_TERMS;
-
-  /** Extension of terms index file */
-  static final String TERMS_INDEX_EXTENSION = "tip";
-  final static String TERMS_INDEX_CODEC_NAME = "BLOCK_TREE_TERMS_INDEX";
-
-  private final IndexOutput out;
+  private final IndexOutput termsOut;
   private final IndexOutput indexOut;
   final int maxDoc;
   final int minItemsInBlock;
@@ -286,8 +249,7 @@ public final class BlockTreeTermsWriter 
    *  sub-blocks) per block will aim to be between
    *  minItemsPerBlock and maxItemsPerBlock, though in some
    *  cases the blocks may be smaller than the min. */
-  public BlockTreeTermsWriter(
-                              SegmentWriteState state,
+  public BlockTreeTermsWriter(SegmentWriteState state,
                               PostingsWriterBase postingsWriter,
                               int minItemsInBlock,
                               int maxItemsInBlock)
@@ -295,47 +257,34 @@ public final class BlockTreeTermsWriter 
   {
     validateSettings(minItemsInBlock, maxItemsInBlock);
 
-    maxDoc = state.segmentInfo.getDocCount();
+    this.maxDoc = state.segmentInfo.getDocCount();
+    this.fieldInfos = state.fieldInfos;
+    this.minItemsInBlock = minItemsInBlock;
+    this.maxItemsInBlock = maxItemsInBlock;
+    this.postingsWriter = postingsWriter;
 
-    final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
-    out = state.directory.createOutput(termsFileName, state.context);
+    final String termsName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_EXTENSION);
+    termsOut = state.directory.createOutput(termsName, state.context);
     boolean success = false;
     IndexOutput indexOut = null;
     try {
-      fieldInfos = state.fieldInfos;
-      this.minItemsInBlock = minItemsInBlock;
-      this.maxItemsInBlock = maxItemsInBlock;
-      writeHeader(out);
-
-      //DEBUG = state.segmentName.equals("_4a");
-
-      final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
-      indexOut = state.directory.createOutput(termsIndexFileName, state.context);
-      writeIndexHeader(indexOut);
+      CodecUtil.writeIndexHeader(termsOut, BlockTreeTermsReader.TERMS_CODEC_NAME, BlockTreeTermsReader.VERSION_CURRENT,
+                                  state.segmentInfo.getId(), state.segmentSuffix);
 
-      this.postingsWriter = postingsWriter;
-      // segment = state.segmentInfo.name;
+      final String indexName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_INDEX_EXTENSION);
+      indexOut = state.directory.createOutput(indexName, state.context);
+      CodecUtil.writeIndexHeader(indexOut, BlockTreeTermsReader.TERMS_INDEX_CODEC_NAME, BlockTreeTermsReader.VERSION_CURRENT,
+                                   state.segmentInfo.getId(), state.segmentSuffix);
 
-      // System.out.println("BTW.init seg=" + state.segmentName);
-
-      postingsWriter.init(out);                          // have consumer write its format/header
+      postingsWriter.init(termsOut, state);                          // have consumer write its format/header
+      
+      this.indexOut = indexOut;
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeWhileHandlingException(out, indexOut);
+        IOUtils.closeWhileHandlingException(termsOut, indexOut);
       }
     }
-    this.indexOut = indexOut;
-  }
-
-  /** Writes the terms file header. */
-  private void writeHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, TERMS_CODEC_NAME, VERSION_CURRENT);   
-  }
-
-  /** Writes the index file header. */
-  private void writeIndexHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, TERMS_INDEX_CODEC_NAME, VERSION_CURRENT); 
   }
 
   /** Writes the terms file trailer. */
@@ -395,7 +344,7 @@ public final class BlockTreeTermsWriter 
   
   static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) {
     assert fp < (1L << 62);
-    return (fp << 2) | (hasTerms ? OUTPUT_FLAG_HAS_TERMS : 0) | (isFloor ? OUTPUT_FLAG_IS_FLOOR : 0);
+    return (fp << 2) | (hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS : 0) | (isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR : 0);
   }
 
   private static class PendingEntry {
@@ -692,7 +641,7 @@ public final class BlockTreeTermsWriter 
 
       assert end > start;
 
-      long startFP = out.getFilePointer();
+      long startFP = termsOut.getFilePointer();
 
       boolean hasFloorLeadLabel = isFloor && floorLeadLabel != -1;
 
@@ -707,7 +656,7 @@ public final class BlockTreeTermsWriter 
         // Last block:
         code |= 1;
       }
-      out.writeVInt(code);
+      termsOut.writeVInt(code);
 
       /*
       if (DEBUG) {
@@ -853,18 +802,18 @@ public final class BlockTreeTermsWriter 
       // search on lookup
 
       // Write suffixes byte[] blob to terms dict output:
-      out.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
-      suffixWriter.writeTo(out);
+      termsOut.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
+      suffixWriter.writeTo(termsOut);
       suffixWriter.reset();
 
       // Write term stats byte[] blob
-      out.writeVInt((int) statsWriter.getFilePointer());
-      statsWriter.writeTo(out);
+      termsOut.writeVInt((int) statsWriter.getFilePointer());
+      statsWriter.writeTo(termsOut);
       statsWriter.reset();
 
       // Write term meta data byte[] blob
-      out.writeVInt((int) metaWriter.getFilePointer());
-      metaWriter.writeTo(out);
+      termsOut.writeVInt((int) metaWriter.getFilePointer());
+      metaWriter.writeTo(termsOut);
       metaWriter.reset();
 
       // if (DEBUG) {
@@ -1019,38 +968,38 @@ public final class BlockTreeTermsWriter 
     boolean success = false;
     try {
       
-      final long dirStart = out.getFilePointer();
+      final long dirStart = termsOut.getFilePointer();
       final long indexDirStart = indexOut.getFilePointer();
 
-      out.writeVInt(fields.size());
+      termsOut.writeVInt(fields.size());
       
       for(FieldMetaData field : fields) {
         //System.out.println("  field " + field.fieldInfo.name + " " + field.numTerms + " terms");
-        out.writeVInt(field.fieldInfo.number);
+        termsOut.writeVInt(field.fieldInfo.number);
         assert field.numTerms > 0;
-        out.writeVLong(field.numTerms);
-        out.writeVInt(field.rootCode.length);
-        out.writeBytes(field.rootCode.bytes, field.rootCode.offset, field.rootCode.length);
+        termsOut.writeVLong(field.numTerms);
+        termsOut.writeVInt(field.rootCode.length);
+        termsOut.writeBytes(field.rootCode.bytes, field.rootCode.offset, field.rootCode.length);
         if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
-          out.writeVLong(field.sumTotalTermFreq);
+          termsOut.writeVLong(field.sumTotalTermFreq);
         }
-        out.writeVLong(field.sumDocFreq);
-        out.writeVInt(field.docCount);
-        out.writeVInt(field.longsSize);
+        termsOut.writeVLong(field.sumDocFreq);
+        termsOut.writeVInt(field.docCount);
+        termsOut.writeVInt(field.longsSize);
         indexOut.writeVLong(field.indexStartFP);
-        writeBytesRef(out, field.minTerm);
-        writeBytesRef(out, field.maxTerm);
+        writeBytesRef(termsOut, field.minTerm);
+        writeBytesRef(termsOut, field.maxTerm);
       }
-      writeTrailer(out, dirStart);
-      CodecUtil.writeFooter(out);
+      writeTrailer(termsOut, dirStart);
+      CodecUtil.writeFooter(termsOut);
       writeIndexTrailer(indexOut, indexDirStart);
       CodecUtil.writeFooter(indexOut);
       success = true;
     } finally {
       if (success) {
-        IOUtils.close(out, indexOut, postingsWriter);
+        IOUtils.close(termsOut, indexOut, postingsWriter);
       } else {
-        IOUtils.closeWhileHandlingException(out, indexOut, postingsWriter);
+        IOUtils.closeWhileHandlingException(termsOut, indexOut, postingsWriter);
       }
     }
   }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java Tue Oct 28 09:19:21 2014
@@ -34,8 +34,10 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.FST;
 
-/** BlockTree's implementation of {@link Terms}. */
-// public for CheckIndex:
+/**
+ * BlockTree's implementation of {@link Terms}.
+ * @lucene.internal
+ */
 public final class FieldReader extends Terms implements Accountable {
 
   private static final long BASE_RAM_BYTES_USED =
@@ -77,7 +79,7 @@ public final class FieldReader extends T
     //   System.out.println("BTTR: seg=" + segment + " field=" + fieldInfo.name + " rootBlockCode=" + rootCode + " divisor=" + indexDivisor);
     // }
 
-    rootBlockFP = (new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong() >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
+    rootBlockFP = (new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong() >>> BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS;
 
     if (indexIn != null) {
       final IndexInput clone = indexIn.clone();
@@ -120,8 +122,8 @@ public final class FieldReader extends T
   }
 
   /** For debugging -- used by CheckIndex too*/
-  // TODO: maybe push this into Terms?
-  public Stats computeStats() throws IOException {
+  @Override
+  public Stats getStats() throws IOException {
     return new SegmentTermsEnum(this).computeBlockStats();
   }
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java Tue Oct 28 09:19:21 2014
@@ -67,7 +67,7 @@ final class IntersectTermsEnum extends T
     this.fr = fr;
     runAutomaton = compiled.runAutomaton;
     compiledAutomaton = compiled;
-    in = fr.parent.in.clone();
+    in = fr.parent.termsIn.clone();
     stack = new IntersectTermsEnumFrame[5];
     for(int idx=0;idx<stack.length;idx++) {
       stack[idx] = new IntersectTermsEnumFrame(this, idx);

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java Tue Oct 28 09:19:21 2014
@@ -145,7 +145,7 @@ final class IntersectTermsEnumFrame {
       // Skip first long -- has redundant fp, hasTerms
       // flag, isFloor flag
       final long code = floorDataReader.readVLong();
-      if ((code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) {
+      if ((code & BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0) {
         numFollowFloorBlocks = floorDataReader.readVInt();
         nextFloorLabel = floorDataReader.readByte() & 0xff;
         // if (DEBUG) System.out.println("    numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java Tue Oct 28 09:19:21 2014
@@ -113,7 +113,7 @@ final class SegmentTermsEnum extends Ter
   // Not private to avoid synthetic access$NNN methods
   void initIndexInput() {
     if (this.in == null) {
-      this.in = fr.parent.in.clone();
+      this.in = fr.parent.termsIn.clone();
     }
   }
 
@@ -238,11 +238,11 @@ final class SegmentTermsEnum extends Ter
   SegmentTermsEnumFrame pushFrame(FST.Arc<BytesRef> arc, BytesRef frameData, int length) throws IOException {
     scratchReader.reset(frameData.bytes, frameData.offset, frameData.length);
     final long code = scratchReader.readVLong();
-    final long fpSeek = code >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
+    final long fpSeek = code >>> BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS;
     final SegmentTermsEnumFrame f = getFrame(1+currentFrame.ord);
-    f.hasTerms = (code & BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0;
+    f.hasTerms = (code & BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS) != 0;
     f.hasTermsOrig = f.hasTerms;
-    f.isFloor = (code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0;
+    f.isFloor = (code & BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0;
     if (f.isFloor) {
       f.setFloorData(scratchReader, frameData);
     }
@@ -370,8 +370,8 @@ final class SegmentTermsEnum extends Ter
         }
         arc = arcs[1+targetUpto];
         assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
-        if (arc.output != BlockTreeTermsWriter.NO_OUTPUT) {
-          output = BlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
+        if (arc.output != BlockTreeTermsReader.NO_OUTPUT) {
+          output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output);
         }
         if (arc.isFinal()) {
           lastFrame = stack[1+lastFrame.ord];
@@ -461,7 +461,7 @@ final class SegmentTermsEnum extends Ter
 
       //term.length = 0;
       targetUpto = 0;
-      currentFrame = pushFrame(arc, BlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
+      currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
     }
 
     // if (DEBUG) {
@@ -517,8 +517,8 @@ final class SegmentTermsEnum extends Ter
         term.setByteAt(targetUpto, (byte) targetLabel);
         // Aggregate output as we go:
         assert arc.output != null;
-        if (arc.output != BlockTreeTermsWriter.NO_OUTPUT) {
-          output = BlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
+        if (arc.output != BlockTreeTermsReader.NO_OUTPUT) {
+          output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output);
         }
 
         // if (DEBUG) {
@@ -528,7 +528,7 @@ final class SegmentTermsEnum extends Ter
 
         if (arc.isFinal()) {
           //if (DEBUG) System.out.println("    arc is final!");
-          currentFrame = pushFrame(arc, BlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
+          currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
           //if (DEBUG) System.out.println("    curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
         }
       }
@@ -612,7 +612,7 @@ final class SegmentTermsEnum extends Ter
 
       int cmp = 0;
 
-      // TOOD: we should write our vLong backwards (MSB
+      // TODO: we should write our vLong backwards (MSB
       // first) to get better sharing from the FST
 
       // First compare up to valid seek frames:
@@ -626,13 +626,13 @@ final class SegmentTermsEnum extends Ter
         }
         arc = arcs[1+targetUpto];
         assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
-        // TOOD: we could save the outputs in local
+        // TODO: we could save the outputs in local
         // byte[][] instead of making new objs ever
         // seek; but, often the FST doesn't have any
         // shared bytes (but this could change if we
         // reverse vLong byte order)
-        if (arc.output != BlockTreeTermsWriter.NO_OUTPUT) {
-          output = BlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
+        if (arc.output != BlockTreeTermsReader.NO_OUTPUT) {
+          output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output);
         }
         if (arc.isFinal()) {
           lastFrame = stack[1+lastFrame.ord];
@@ -717,7 +717,7 @@ final class SegmentTermsEnum extends Ter
 
       //term.length = 0;
       targetUpto = 0;
-      currentFrame = pushFrame(arc, BlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
+      currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
     }
 
     //if (DEBUG) {
@@ -773,8 +773,8 @@ final class SegmentTermsEnum extends Ter
         arc = nextArc;
         // Aggregate output as we go:
         assert arc.output != null;
-        if (arc.output != BlockTreeTermsWriter.NO_OUTPUT) {
-          output = BlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
+        if (arc.output != BlockTreeTermsReader.NO_OUTPUT) {
+          output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output);
         }
 
         //if (DEBUG) {
@@ -784,7 +784,7 @@ final class SegmentTermsEnum extends Ter
 
         if (arc.isFinal()) {
           //if (DEBUG) System.out.println("    arc is final!");
-          currentFrame = pushFrame(arc, BlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
+          currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
           //if (DEBUG) System.out.println("    curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
         }
       }
@@ -831,9 +831,9 @@ final class SegmentTermsEnum extends Ter
         assert f != null;
         final BytesRef prefix = new BytesRef(term.get().bytes, 0, f.prefix);
         if (f.nextEnt == -1) {
-          out.println("    frame " + (isSeekFrame ? "(seek)" : "(next)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
+          out.println("    frame " + (isSeekFrame ? "(seek)" : "(next)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<< BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR:0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
         } else {
-          out.println("    frame " + (isSeekFrame ? "(seek, loaded)" : "(next, loaded)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + " nextEnt=" + f.nextEnt + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " lastSubFP=" + f.lastSubFP + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
+          out.println("    frame " + (isSeekFrame ? "(seek, loaded)" : "(next, loaded)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + " nextEnt=" + f.nextEnt + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<< BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR:0)) + " lastSubFP=" + f.lastSubFP + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
         }
         if (fr.index != null) {
           assert !isSeekFrame || f.arc != null: "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc;
@@ -848,7 +848,7 @@ final class SegmentTermsEnum extends Ter
           } else if (isSeekFrame && !f.isFloor) {
             final ByteArrayDataInput reader = new ByteArrayDataInput(output.bytes, output.offset, output.length);
             final long codeOrig = reader.readVLong();
-            final long code = (f.fp << BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) | (f.hasTerms ? BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) | (f.isFloor ? BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0);
+            final long code = (f.fp << BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS) | (f.hasTerms ? BlockTreeTermsReader.OUTPUT_FLAG_HAS_TERMS:0) | (f.isFloor ? BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR:0);
             if (codeOrig != code) {
               out.println("      broken seek state: output code=" + codeOrig + " doesn't match frame code=" + code);
               throw new RuntimeException("seek state is broken");

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java Tue Oct 28 09:19:21 2014
@@ -22,14 +22,15 @@ import java.io.PrintStream;
 import java.io.UnsupportedEncodingException;
 import java.util.Locale;
 
-import org.apache.lucene.codecs.PostingsBaseFormat;
+import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 /**
  * BlockTree statistics for a single field 
- * returned by {@link FieldReader#computeStats()}.
+ * returned by {@link FieldReader#getStats()}.
+ * @lucene.internal
  */
 public class Stats {
   /** How many nodes in the index FST. */
@@ -81,11 +82,11 @@ public class Stats {
   public long totalBlockSuffixBytes;
 
   /** Total number of bytes used to store term stats (not
-   *  including what the {@link PostingsBaseFormat}
+   *  including what the {@link PostingsReaderBase}
    *  stores. */
   public long totalBlockStatsBytes;
 
-  /** Total bytes stored by the {@link PostingsBaseFormat},
+  /** Total bytes stored by the {@link PostingsReaderBase},
    *  plus the other few vInts stored in the frame. */
   public long totalBlockOtherBytes;
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/package.html?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/package.html (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/package.html Tue Oct 28 09:19:21 2014
@@ -27,7 +27,7 @@ This terms dictionary organizes all term
 shared prefix, such that each block has enough terms, and then stores
 the prefix trie in memory as an FST as the index structure.  It allows
 you to plug in your own {@link
-org.apache.lucene.codecs.PostingsBaseFormat} to implement the
+org.apache.lucene.codecs.PostingsWriterBase} to implement the
 postings.
 </p>
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsFormat.java Tue Oct 28 09:19:21 2014
@@ -64,7 +64,7 @@ public class CompressingStoredFieldsForm
    * <p>
    * <code>formatName</code> is the name of the format. This name will be used
    * in the file formats to perform
-   * {@link CodecUtil#checkSegmentHeader codec header checks}.
+   * {@link CodecUtil#checkIndexHeader codec header checks}.
    * <p>
    * <code>segmentSuffix</code> is the segment suffix. This suffix is added to 
    * the result file name only if it's not the empty string.

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsReader.java Tue Oct 28 09:19:21 2014
@@ -118,8 +118,8 @@ public final class CompressingStoredFiel
       Throwable priorE = null;
       try {
         final String codecNameIdx = formatName + CODEC_SFX_IDX;
-        version = CodecUtil.checkSegmentHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
-        assert CodecUtil.segmentHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
+        version = CodecUtil.checkIndexHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
+        assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
         indexReader = new CompressingStoredFieldsIndexReader(indexStream, si);
         maxPointer = indexStream.readVLong();
       } catch (Throwable exception) {
@@ -141,11 +141,11 @@ public final class CompressingStoredFiel
         throw new CorruptIndexException("Invalid fieldsStream maxPointer (file truncated?): maxPointer=" + maxPointer + ", length=" + fieldsStream.length(), fieldsStream);
       }
       final String codecNameDat = formatName + CODEC_SFX_DAT;
-      final int fieldsVersion = CodecUtil.checkSegmentHeader(fieldsStream, codecNameDat, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
+      final int fieldsVersion = CodecUtil.checkIndexHeader(fieldsStream, codecNameDat, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
       if (version != fieldsVersion) {
         throw new CorruptIndexException("Version mismatch between stored fields index and data: " + version + " != " + fieldsVersion, fieldsStream);
       }
-      assert CodecUtil.segmentHeaderLength(codecNameDat, segmentSuffix) == fieldsStream.getFilePointer();
+      assert CodecUtil.indexHeaderLength(codecNameDat, segmentSuffix) == fieldsStream.getFilePointer();
 
       chunkSize = fieldsStream.readVInt();
       packedIntsVersion = fieldsStream.readVInt();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java Tue Oct 28 09:19:21 2014
@@ -29,13 +29,9 @@ import org.apache.lucene.index.CorruptIn
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.SegmentReader;
 import org.apache.lucene.index.StorableField;
-import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -118,10 +114,10 @@ public final class CompressingStoredFiel
 
       final String codecNameIdx = formatName + CODEC_SFX_IDX;
       final String codecNameDat = formatName + CODEC_SFX_DAT;
-      CodecUtil.writeSegmentHeader(indexStream, codecNameIdx, VERSION_CURRENT, si.getId(), segmentSuffix);
-      CodecUtil.writeSegmentHeader(fieldsStream, codecNameDat, VERSION_CURRENT, si.getId(), segmentSuffix);
-      assert CodecUtil.segmentHeaderLength(codecNameDat, segmentSuffix) == fieldsStream.getFilePointer();
-      assert CodecUtil.segmentHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
+      CodecUtil.writeIndexHeader(indexStream, codecNameIdx, VERSION_CURRENT, si.getId(), segmentSuffix);
+      CodecUtil.writeIndexHeader(fieldsStream, codecNameDat, VERSION_CURRENT, si.getId(), segmentSuffix);
+      assert CodecUtil.indexHeaderLength(codecNameDat, segmentSuffix) == fieldsStream.getFilePointer();
+      assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
 
       indexWriter = new CompressingStoredFieldsIndexWriter(indexStream);
       indexStream = null;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsFormat.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsFormat.java Tue Oct 28 09:19:21 2014
@@ -46,7 +46,7 @@ public class CompressingTermVectorsForma
    * <p>
    * <code>formatName</code> is the name of the format. This name will be used
    * in the file formats to perform
-   * {@link CodecUtil#checkSegmentHeader codec header checks}.
+   * {@link CodecUtil#checkIndexHeader codec header checks}.
    * <p>
    * The <code>compressionMode</code> parameter allows you to choose between
    * compression algorithms that have various compression and decompression

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java Tue Oct 28 09:19:21 2014
@@ -114,8 +114,8 @@ public final class CompressingTermVector
       Throwable priorE = null;
       try {
         final String codecNameIdx = formatName + CODEC_SFX_IDX;
-        version = CodecUtil.checkSegmentHeader(input, codecNameIdx, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
-        assert CodecUtil.segmentHeaderLength(codecNameIdx, segmentSuffix) == input.getFilePointer();
+        version = CodecUtil.checkIndexHeader(input, codecNameIdx, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
+        assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == input.getFilePointer();
         indexReader = new CompressingStoredFieldsIndexReader(input, si);
         input.readVLong(); // the end of the data file
       } catch (Throwable exception) {
@@ -133,11 +133,11 @@ public final class CompressingTermVector
       final String vectorsStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION);
       vectorsStream = d.openInput(vectorsStreamFN, context);
       final String codecNameDat = formatName + CODEC_SFX_DAT;
-      int version2 = CodecUtil.checkSegmentHeader(vectorsStream, codecNameDat, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
+      int version2 = CodecUtil.checkIndexHeader(vectorsStream, codecNameDat, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
       if (version != version2) {
         throw new CorruptIndexException("Version mismatch between stored fields index and data: " + version + " != " + version2, vectorsStream);
       }
-      assert CodecUtil.segmentHeaderLength(codecNameDat, segmentSuffix) == vectorsStream.getFilePointer();
+      assert CodecUtil.indexHeaderLength(codecNameDat, segmentSuffix) == vectorsStream.getFilePointer();
       
       long pos = vectorsStream.getFilePointer();
       // NOTE: data file is too costly to verify checksum against all the bytes on open,

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java Tue Oct 28 09:19:21 2014
@@ -32,11 +32,8 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.SegmentReader;
 import org.apache.lucene.store.BufferedChecksumIndexInput;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.DataInput;
@@ -231,10 +228,10 @@ public final class CompressingTermVector
 
       final String codecNameIdx = formatName + CODEC_SFX_IDX;
       final String codecNameDat = formatName + CODEC_SFX_DAT;
-      CodecUtil.writeSegmentHeader(indexStream, codecNameIdx, VERSION_CURRENT, si.getId(), segmentSuffix);
-      CodecUtil.writeSegmentHeader(vectorsStream, codecNameDat, VERSION_CURRENT, si.getId(), segmentSuffix);
-      assert CodecUtil.segmentHeaderLength(codecNameDat, segmentSuffix) == vectorsStream.getFilePointer();
-      assert CodecUtil.segmentHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
+      CodecUtil.writeIndexHeader(indexStream, codecNameIdx, VERSION_CURRENT, si.getId(), segmentSuffix);
+      CodecUtil.writeIndexHeader(vectorsStream, codecNameDat, VERSION_CURRENT, si.getId(), segmentSuffix);
+      assert CodecUtil.indexHeaderLength(codecNameDat, segmentSuffix) == vectorsStream.getFilePointer();
+      assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix) == indexStream.getFilePointer();
 
       indexWriter = new CompressingStoredFieldsIndexWriter(indexStream);
       indexStream = null;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java Tue Oct 28 09:19:21 2014
@@ -106,7 +106,7 @@ public class Lucene50Codec extends Codec
   /** Returns the postings format that should be used for writing 
    *  new segments of <code>field</code>.
    *  
-   *  The default implementation always returns "Lucene41"
+   *  The default implementation always returns "Lucene50"
    */
   public PostingsFormat getPostingsFormatForField(String field) {
     return defaultFormat;
@@ -115,7 +115,7 @@ public class Lucene50Codec extends Codec
   /** Returns the docvalues format that should be used for writing 
    *  new segments of <code>field</code>.
    *  
-   *  The default implementation always returns "Lucene410"
+   *  The default implementation always returns "Lucene50"
    */
   public DocValuesFormat getDocValuesFormatForField(String field) {
     return defaultDVFormat;
@@ -126,8 +126,8 @@ public class Lucene50Codec extends Codec
     return docValuesFormat;
   }
 
-  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
-  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene410");
+  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
+  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene50");
 
   private final NormsFormat normsFormat = new Lucene50NormsFormat();
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java Tue Oct 28 09:19:21 2014
@@ -46,7 +46,7 @@ import org.apache.lucene.store.IndexOutp
  *   <li>Compound (.cfs) --&gt; Header, FileData <sup>FileCount</sup>, Footer</li>
  *   <li>Compound Entry Table (.cfe) --&gt; Header, FileCount, &lt;FileName,
  *       DataOffset, DataLength&gt; <sup>FileCount</sup></li>
- *   <li>Header --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
+ *   <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  *   <li>FileCount --&gt; {@link DataOutput#writeVInt VInt}</li>
  *   <li>DataOffset,DataLength,Checksum --&gt; {@link DataOutput#writeLong UInt64}</li>
  *   <li>FileName --&gt; {@link DataOutput#writeString String}</li>
@@ -79,8 +79,8 @@ public final class Lucene50CompoundForma
     
     try (IndexOutput data =    dir.createOutput(dataFile, context);
          IndexOutput entries = dir.createOutput(entriesFile, context)) {
-      CodecUtil.writeSegmentHeader(data,    DATA_CODEC, VERSION_CURRENT, si.getId(), "");
-      CodecUtil.writeSegmentHeader(entries, ENTRY_CODEC, VERSION_CURRENT, si.getId(), "");
+      CodecUtil.writeIndexHeader(data,    DATA_CODEC, VERSION_CURRENT, si.getId(), "");
+      CodecUtil.writeIndexHeader(entries, ENTRY_CODEC, VERSION_CURRENT, si.getId(), "");
       
       // write number of files
       entries.writeVInt(files.size());

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java Tue Oct 28 09:19:21 2014
@@ -71,7 +71,7 @@ final class Lucene50CompoundReader exten
     boolean success = false;
     handle = directory.openInput(dataFileName, context);
     try {
-      CodecUtil.checkSegmentHeader(handle, Lucene50CompoundFormat.DATA_CODEC, version, version, si.getId(), "");
+      CodecUtil.checkIndexHeader(handle, Lucene50CompoundFormat.DATA_CODEC, version, version, si.getId(), "");
       
       // NOTE: data file is too costly to verify checksum against all the bytes on open,
       // but for now we at least verify proper structure of the checksum footer: which looks
@@ -93,7 +93,7 @@ final class Lucene50CompoundReader exten
     try (ChecksumIndexInput entriesStream = dir.openChecksumInput(entriesFileName, IOContext.READONCE)) {
       Throwable priorE = null;
       try {
-        version = CodecUtil.checkSegmentHeader(entriesStream, Lucene50CompoundFormat.ENTRY_CODEC, 
+        version = CodecUtil.checkIndexHeader(entriesStream, Lucene50CompoundFormat.ENTRY_CODEC, 
                                                               Lucene50CompoundFormat.VERSION_START, 
                                                               Lucene50CompoundFormat.VERSION_CURRENT, segmentID, "");
         final int numEntries = entriesStream.readVInt();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java Tue Oct 28 09:19:21 2014
@@ -46,7 +46,7 @@ import org.apache.lucene.store.IndexOutp
  * FieldBits,DocValuesBits,DocValuesGen,Attributes&gt; <sup>FieldsCount</sup>,Footer</p>
  * <p>Data types:
  * <ul>
- *   <li>Header --&gt; {@link CodecUtil#checkSegmentHeader SegmentHeader}</li>
+ *   <li>Header --&gt; {@link CodecUtil#checkIndexHeader IndexHeader}</li>
  *   <li>FieldsCount --&gt; {@link DataOutput#writeVInt VInt}</li>
  *   <li>FieldName --&gt; {@link DataOutput#writeString String}</li>
  *   <li>FieldBits, IndexOptions, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
@@ -114,9 +114,9 @@ public final class Lucene50FieldInfosFor
       Throwable priorE = null;
       FieldInfo infos[] = null;
       try {
-        CodecUtil.checkSegmentHeader(input, CODEC_NAME, 
-                                     FORMAT_START, 
-                                     FORMAT_CURRENT,
+        CodecUtil.checkIndexHeader(input, Lucene50FieldInfosFormat.CODEC_NAME, 
+                                     Lucene50FieldInfosFormat.FORMAT_START, 
+                                     Lucene50FieldInfosFormat.FORMAT_CURRENT,
                                      segmentInfo.getId(), segmentSuffix);
         
         final int size = input.readVInt(); //read in the size
@@ -251,7 +251,7 @@ public final class Lucene50FieldInfosFor
   public void write(Directory directory, SegmentInfo segmentInfo, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, EXTENSION);
     try (IndexOutput output = directory.createOutput(fileName, context)) {
-      CodecUtil.writeSegmentHeader(output, CODEC_NAME, FORMAT_CURRENT, segmentInfo.getId(), segmentSuffix);
+      CodecUtil.writeIndexHeader(output, Lucene50FieldInfosFormat.CODEC_NAME, Lucene50FieldInfosFormat.FORMAT_CURRENT, segmentInfo.getId(), segmentSuffix);
       output.writeVInt(infos.size());
       for (FieldInfo fi : infos) {
         fi.checkConsistency();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java Tue Oct 28 09:19:21 2014
@@ -41,10 +41,9 @@ import org.apache.lucene.util.MutableBit
  * deletions.</p>
  * <p>Although per-segment, this file is maintained exterior to compound segment
  * files.</p>
- * <p>Deletions (.liv) --&gt; SegmentHeader,Generation,Bits</p>
+ * <p>Deletions (.liv) --&gt; IndexHeader,Generation,Bits</p>
  * <ul>
- *   <li>SegmentHeader --&gt; {@link CodecUtil#writeSegmentHeader SegmentHeader}</li>
- *   <li>Generation --&gt; {@link DataOutput#writeLong Int64}
+ *   <li>SegmentHeader --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  *   <li>Bits --&gt; &lt;{@link DataOutput#writeLong Int64}&gt; <sup>LongCount</sup></li>
  * </ul>
  */
@@ -85,11 +84,8 @@ public final class Lucene50LiveDocsForma
     try (ChecksumIndexInput input = dir.openChecksumInput(name, context)) {
       Throwable priorE = null;
       try {
-        CodecUtil.checkSegmentHeader(input, CODEC_NAME, VERSION_START, VERSION_CURRENT, info.info.getId(), "");
-        long filegen = input.readLong();
-        if (gen != filegen) {
-          throw new CorruptIndexException("file mismatch, expected generation=" + gen + ", got=" + filegen, input);
-        }
+        CodecUtil.checkIndexHeader(input, CODEC_NAME, VERSION_START, VERSION_CURRENT, 
+                                     info.info.getId(), Long.toString(gen, Character.MAX_RADIX));
         long data[] = new long[FixedBitSet.bits2words(length)];
         for (int i = 0; i < data.length; i++) {
           data[i] = input.readLong();
@@ -120,8 +116,7 @@ public final class Lucene50LiveDocsForma
     }
     long data[] = fbs.getBits();
     try (IndexOutput output = dir.createOutput(name, context)) {
-      CodecUtil.writeSegmentHeader(output, CODEC_NAME, VERSION_CURRENT, info.info.getId(), "");
-      output.writeLong(gen);
+      CodecUtil.writeIndexHeader(output, CODEC_NAME, VERSION_CURRENT, info.info.getId(), Long.toString(gen, Character.MAX_RADIX));
       for (int i = 0; i < data.length; i++) {
         output.writeLong(data[i]);
       }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsConsumer.java Tue Oct 28 09:19:21 2014
@@ -47,6 +47,7 @@ class Lucene50NormsConsumer extends Norm
   static final byte CONST_COMPRESSED = 2;
   static final byte UNCOMPRESSED = 3;
   static final byte INDIRECT = 4;
+  static final byte PATCHED = 5;
   static final int BLOCK_SIZE = 1 << 14;
   
   // threshold for indirect encoding, computed as 1 - 1/log2(maxint)
@@ -61,10 +62,10 @@ class Lucene50NormsConsumer extends Norm
     try {
       String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.createOutput(dataName, state.context);
-      CodecUtil.writeSegmentHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      CodecUtil.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
       String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
       meta = state.directory.createOutput(metaName, state.context);
-      CodecUtil.writeSegmentHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
       success = true;
     } finally {
       if (!success) {
@@ -82,6 +83,11 @@ class Lucene50NormsConsumer extends Norm
 
   @Override
   public void addNormsField(FieldInfo field, Iterable<Number> values) throws IOException {
+    writeNormsField(field, values, 0);
+  }
+  
+  private void writeNormsField(FieldInfo field, Iterable<Number> values, int level) throws IOException {
+    assert level <= 1; // we only "recurse" once in the indirect case
     meta.writeVInt(field.number);
     long minValue = Long.MAX_VALUE;
     long maxValue = Long.MIN_VALUE;
@@ -89,16 +95,12 @@ class Lucene50NormsConsumer extends Norm
     NormMap uniqueValues = new NormMap();
     
     int count = 0;
-    int missingCount = 0;
     
     for (Number nv : values) {
       if (nv == null) {
         throw new IllegalStateException("illegal norms data for field " + field.name + ", got null for value: " + count);
       }
       final long v = nv.longValue();
-      if (v == 0) {
-        missingCount++;
-      }
       
       minValue = Math.min(minValue, v);
       maxValue = Math.max(maxValue, v);
@@ -115,9 +117,15 @@ class Lucene50NormsConsumer extends Norm
     if (uniqueValues != null && uniqueValues.size == 1) {
       // 0 bpv
       addConstant(minValue);
-    } else if (count > 256 && missingCount > count * INDIRECT_THRESHOLD) {
-      // sparse encoding
-      addIndirect(field, values, count, missingCount);
+    } else if (level == 0 && count > 256 && uniqueValues != null && uniqueValues.maxFreq() > count * INDIRECT_THRESHOLD) {
+      long commonValue = uniqueValues.getDecodeTable()[uniqueValues.maxOrd()];
+      if (commonValue == 0) {
+        // if the common value is missing, don't waste RAM on a bitset, since we won't be searching those docs
+        addIndirect(field, values, count, uniqueValues);
+      } else {
+        // otherwise, write a sparse bitset, where 1 indicates 'uncommon value'.
+        addPatched(field, values, count, uniqueValues);
+      }
     } else if (uniqueValues != null) {
       // small number of unique values: this is the typical case:
       FormatAndBits compression = fastestFormatAndBits(uniqueValues.size-1);
@@ -200,10 +208,65 @@ class Lucene50NormsConsumer extends Norm
     writer.finish();
   }
   
-  private void addIndirect(FieldInfo field, final Iterable<Number> values, int count, int missingCount) throws IOException {
-    meta.writeVInt(count - missingCount);
+  // encodes only uncommon values in a sparse bitset
+  // access is constant time, and the common case is predictable
+  // exceptions nest either to CONST (if there are only 2 values), or INDIRECT (if there are > 2 values)
+  private void addPatched(FieldInfo field, final Iterable<Number> values, int count, NormMap uniqueValues) throws IOException {
+    final long decodeTable[] = uniqueValues.getDecodeTable();
+    int commonCount = uniqueValues.maxFreq();
+    final long commonValue = decodeTable[uniqueValues.maxOrd()];
+    
+    meta.writeVInt(count - commonCount);
+    meta.writeByte(PATCHED);
+    meta.writeLong(data.getFilePointer());
+    
+    // write docs with value
+    writeDocsWithValue(values, commonValue);
+    
+    // write exceptions: only two cases make sense
+    // bpv = 1 (folded into sparse bitset already)
+    // bpv > 1 (add indirect exception table)
+    meta.writeVInt(field.number);
+    if (uniqueValues.size == 2) {
+      // special case: implicit in bitset
+      int otherOrd = uniqueValues.maxOrd() == 0 ? 1 : 0;
+      addConstant(decodeTable[otherOrd]);
+    } else {
+      // exception table
+      addIndirect(field, values, count, uniqueValues);
+    }
+  }
+  
+  // encodes values as sparse array: keys[] and values[]
+  // access is log(N) where N = keys.length (slow!)
+  // so this is only appropriate as an exception table for patched, or when common value is 0 (wont be accessed by searching) 
+  private void addIndirect(FieldInfo field, final Iterable<Number> values, int count, NormMap uniqueValues) throws IOException {
+    int commonCount = uniqueValues.maxFreq();
+    final long commonValue = uniqueValues.getDecodeTable()[uniqueValues.maxOrd()];
+    
+    meta.writeVInt(count - commonCount);
     meta.writeByte(INDIRECT);
     meta.writeLong(data.getFilePointer());
+    
+    // write docs with value
+    writeDocsWithValue(values, commonValue);
+    
+    // write actual values
+    writeNormsField(field, new Iterable<Number>() {
+      @Override
+      public Iterator<Number> iterator() {
+        return new FilterIterator<Number,Number>(values.iterator()) {
+          @Override
+          protected boolean predicateFunction(Number value) {
+            return value.longValue() != commonValue;
+          }
+        };
+      }
+    }, 1);
+  }
+  
+  private void writeDocsWithValue(final Iterable<Number> values, long commonValue) throws IOException {
+    data.writeLong(commonValue);
     data.writeVInt(PackedInts.VERSION_CURRENT);
     data.writeVInt(BLOCK_SIZE);
     
@@ -212,25 +275,12 @@ class Lucene50NormsConsumer extends Norm
     int doc = 0;
     for (Number n : values) {
       long v = n.longValue();
-      if (v != 0) {
+      if (v != commonValue) {
         writer.add(doc);
       }
       doc++;
     }
     writer.finish();
-    
-    // write actual values
-    addNormsField(field, new Iterable<Number>() {
-      @Override
-      public Iterator<Number> iterator() {
-        return new FilterIterator<Number,Number>(values.iterator()) {
-          @Override
-          protected boolean predicateFunction(Number value) {
-            return value.longValue() != 0;
-          }
-        };
-      }
-    });
   }
   
   @Override
@@ -259,6 +309,7 @@ class Lucene50NormsConsumer extends Norm
   static class NormMap {
     // we use short: at most we will add 257 values to this map before its rejected as too big above.
     final short[] singleByteRange = new short[256];
+    final int[] freqs = new int[257];
     final Map<Long,Short> other = new HashMap<Long,Short>();
     int size;
     
@@ -273,18 +324,24 @@ class Lucene50NormsConsumer extends Norm
         int index = (int) (l + 128);
         short previous = singleByteRange[index];
         if (previous < 0) {
-          singleByteRange[index] = (short) size;
+          short slot = (short) size;
+          singleByteRange[index] = slot;
+          freqs[slot]++;
           size++;
           return true;
         } else {
+          freqs[previous]++;
           return false;
         }
       } else {
-        if (!other.containsKey(l)) {
+        Short previous = other.get(l);
+        if (previous == null) {
+          freqs[size]++;
           other.put(l, (short)size);
           size++;
           return true;
         } else {
+          freqs[previous]++;
           return false;
         }
       }
@@ -315,5 +372,35 @@ class Lucene50NormsConsumer extends Norm
       }
       return decode;
     }
+    
+    // TODO: if we need more complicated frequency-driven optos, maybe add 'finish' to this api
+    // and sort all ords by frequency. we could then lower BPV and waste a value to represent 'patched',
+    
+    /** retrieves frequency table for items (indexed by ordinal) */
+    public int[] getFreqs() {
+      return freqs;
+    }
+    
+    /** sugar: returns max value over getFreqs() */
+    public int maxFreq() {
+      int max = 0;
+      for (int i = 0; i < size; i++) {
+        max = Math.max(max, freqs[i]);
+      }
+      return max;
+    }
+    
+    /** sugar: returns ordinal with maxFreq() */
+    public int maxOrd() {
+      long max = 0;
+      int maxOrd = 0;
+      for (int i = 0; i < size; i++) {
+        if (freqs[i] > max) {
+          max = freqs[i];
+          maxOrd = i;
+        }
+      }
+      return maxOrd;
+    }
   }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsFormat.java Tue Oct 28 09:19:21 2014
@@ -28,6 +28,7 @@ import org.apache.lucene.index.SegmentWr
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.SmallFloat;
 import org.apache.lucene.util.packed.BlockPackedWriter;
+import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
 import org.apache.lucene.util.packed.PackedInts;
 
 /**
@@ -50,6 +51,9 @@ import org.apache.lucene.util.packed.Pac
  *    <li>Indirect: when norms are extremely sparse, missing values are omitted.
  *        Access to an individual value is slower, but missing norm values are never accessed
  *        by search code.
+ *    <li>Patched: when a single norm value dominates, a sparse bitset encodes docs with exceptions,
+ *        so that access to the common value is still very fast. outliers fall thru to an exception 
+ *        handling mechanism (Indirect or Constant).
  * </ul>
  * <p>
  * Files:
@@ -64,7 +68,7 @@ import org.apache.lucene.util.packed.Pac
  *      Norms data (.nvd)</p>
  *   <p>Norms metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
  *   <ul>
- *     <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  *     <li>Entry --&gt; FieldNumber,Type,Offset</li>
  *     <li>FieldNumber --&gt; {@link DataOutput#writeVInt vInt}</li>
  *     <li>Type --&gt; {@link DataOutput#writeByte Byte}</li>
@@ -81,20 +85,24 @@ import org.apache.lucene.util.packed.Pac
  *             a lookup table of unique values is written, followed by the ordinal for each document.
  *         <li>2 --&gt; constant. When there is a single value for the entire field.
  *         <li>3 --&gt; uncompressed: Values written as a simple byte[].
- *         <li>4 --&gt; indirect. Only documents with a value are written with a sparse encoding.
+ *         <li>4 --&gt; indirect. Only documents with a value are written with monotonic compression. a nested
+ *             entry for the same field will follow for the exception handler.
+ *         <li>5 --&gt; patched. Encoded the same as indirect.
  *      </ul>
  *   <li><a name="nvd" id="nvd"></a>
  *   <p>The Norms data or .nvd file.</p>
  *   <p>For each Norms field, this stores the actual per-document data (the heavy-lifting)</p>
- *   <p>Norms data (.nvd) --&gt; Header,&lt;Uncompressed | TableCompressed | DeltaCompressed&gt;<sup>NumFields</sup>,Footer</p>
+ *   <p>Norms data (.nvd) --&gt; Header,&lt;Uncompressed | TableCompressed | DeltaCompressed | MonotonicCompressed &gt;<sup>NumFields</sup>,Footer</p>
  *   <ul>
- *     <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *     <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  *     <li>Uncompressed --&gt;  {@link DataOutput#writeByte Byte}<sup>maxDoc</sup></li>
  *     <li>TableCompressed --&gt; PackedIntsVersion,Table,BitPackedData</li>
  *     <li>Table --&gt; TableSize, {@link DataOutput#writeLong int64}<sup>TableSize</sup></li>
  *     <li>BitpackedData --&gt; {@link PackedInts}</li>
  *     <li>DeltaCompressed --&gt; PackedIntsVersion,BlockSize,DeltaCompressedData</li>
  *     <li>DeltaCompressedData --&gt; {@link BlockPackedWriter BlockPackedWriter(blockSize=16k)}</li>
+ *     <li>MonotonicCompressed --&gt; PackedIntsVersion,BlockSize,MonotonicCompressedData</li>
+ *     <li>MonotonicCompressedData --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedWriter(blockSize=16k)}</li>
  *     <li>PackedIntsVersion,BlockSize,TableSize --&gt; {@link DataOutput#writeVInt vInt}</li>
  *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  *   </ul>

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java?rev=1634823&r1=1634822&r2=1634823&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50NormsProducer.java Tue Oct 28 09:19:21 2014
@@ -37,6 +37,7 @@ import org.apache.lucene.util.Accountabl
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.SparseFixedBitSet;
 import org.apache.lucene.util.packed.BlockPackedReader;
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
 import org.apache.lucene.util.packed.PackedInts;
@@ -48,6 +49,7 @@ import static org.apache.lucene.codecs.l
 import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.TABLE_COMPRESSED;
 import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.UNCOMPRESSED;
 import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.INDIRECT;
+import static org.apache.lucene.codecs.lucene50.Lucene50NormsConsumer.PATCHED;
 
 /**
  * Reader for {@link Lucene50NormsFormat}
@@ -63,6 +65,7 @@ class Lucene50NormsProducer extends Norm
   
   private final AtomicLong ramBytesUsed;
   private final AtomicInteger activeCount = new AtomicInteger();
+  private final int maxDoc;
   
   private final boolean merging;
   
@@ -75,11 +78,13 @@ class Lucene50NormsProducer extends Norm
     instancesInfo.putAll(original.instancesInfo);
     ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
     activeCount.set(original.activeCount.get());
+    maxDoc = original.maxDoc;
     merging = true;
   }
     
   Lucene50NormsProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     merging = false;
+    maxDoc = state.segmentInfo.getDocCount();
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
     ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
     int version = -1;
@@ -88,7 +93,7 @@ class Lucene50NormsProducer extends Norm
     try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
       Throwable priorE = null;
       try {
-        version = CodecUtil.checkSegmentHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+        version = CodecUtil.checkIndexHeader(in, metaCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
         readFields(in, state.fieldInfos);
       } catch (Throwable exception) {
         priorE = exception;
@@ -101,7 +106,7 @@ class Lucene50NormsProducer extends Norm
     this.data = state.directory.openInput(dataName, state.context);
     boolean success = false;
     try {
-      final int version2 = CodecUtil.checkSegmentHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
+      final int version2 = CodecUtil.checkIndexHeader(data, dataCodec, VERSION_START, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
       if (version != version2) {
         throw new CorruptIndexException("Format versions mismatch: meta=" + version + ",data=" + version2, data);
       }
@@ -146,6 +151,7 @@ class Lucene50NormsProducer extends Norm
       case TABLE_COMPRESSED:
       case DELTA_COMPRESSED:
         break;
+      case PATCHED:
       case INDIRECT:
         if (meta.readVInt() != info.number) {
           throw new CorruptIndexException("indirect norms entry for field: " + info.name + " is corrupt", meta);
@@ -254,6 +260,7 @@ class Lucene50NormsProducer extends Norm
       }
       case INDIRECT: {
         data.seek(entry.offset);
+        final long common = data.readLong();
         int packedIntsVersion = data.readVInt();
         int blockSize = data.readVInt();
         final MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, false);
@@ -279,7 +286,34 @@ class Lucene50NormsProducer extends Norm
                 return values.get(mid);
               }
             }
-            return 0;
+            return common;
+          }
+        };
+        break;
+      }
+      case PATCHED: {
+        data.seek(entry.offset);
+        final long common = data.readLong();
+        int packedIntsVersion = data.readVInt();
+        int blockSize = data.readVInt();
+        MonotonicBlockPackedReader live = MonotonicBlockPackedReader.of(data, packedIntsVersion, blockSize, entry.count, true);
+        final SparseFixedBitSet set = new SparseFixedBitSet(maxDoc);
+        for (int i = 0; i < live.size(); i++) {
+          int doc = (int) live.get(i);
+          set.set(doc);
+        }
+        LoadedNorms nestedInstance = loadNorms(entry.nested);
+        instance.ramBytesUsed = set.ramBytesUsed() + nestedInstance.ramBytesUsed;
+        instance.info = Accountables.namedAccountable("patched -> " + nestedInstance.info, instance.ramBytesUsed);
+        final NumericDocValues values = nestedInstance.norms;
+        instance.norms = new NumericDocValues() {
+          @Override
+          public long get(int docID) {
+            if (set.get(docID)) {
+              return values.get(docID);
+            } else {
+              return common;
+            }
           }
         };
         break;