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 2014/09/28 00:44:45 UTC

svn commit: r1628019 [1/3] - in /lucene/dev/branches/lucene5969/lucene: backward-codecs/src/java/org/apache/lucene/codecs/lucene41/ backward-codecs/src/java/org/apache/lucene/codecs/lucene42/ backward-codecs/src/java/org/apache/lucene/codecs/lucene45/ ...

Author: rmuir
Date: Sat Sep 27 22:44:44 2014
New Revision: 1628019

URL: http://svn.apache.org/r1628019
Log:
LUCENE-5969: copy over cruft for back compat

Added:
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java
      - copied, changed from r1627946, lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexReader.java   (with props)
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsReader.java   (with props)
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java
      - copied, changed from r1627946, lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsReader.java   (with props)
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWStoredFieldsFormat.java   (with props)
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexWriter.java   (with props)
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java   (with props)
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41StoredFieldsFormat.java   (with props)
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWTermVectorsFormat.java   (with props)
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsWriter.java   (with props)
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/TestLucene42TermVectorsFormat.java   (with props)
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java   (with props)
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java   (with props)
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50StoredFieldsFormat.java   (with props)
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50TermVectorsFormat.java   (with props)
Removed:
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene42/
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestLucene41PostingsFormat.java
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/codecs/lucene41/TestLucene41StoredFieldsFormat.java
Modified:
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49Codec.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWCodec.java
    lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
    lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html
    lucene/dev/branches/lucene5969/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java

Copied: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java (from r1627946, lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java?p2=lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java&p1=lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java&r1=1627946&r2=1628019&rev=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java Sat Sep 27 22:44:44 2014
@@ -17,108 +17,39 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
-import org.apache.lucene.codecs.CodecUtil;
+import java.io.IOException;
+
 import org.apache.lucene.codecs.StoredFieldsFormat;
-import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
-import org.apache.lucene.codecs.compressing.CompressingStoredFieldsIndexWriter;
+import org.apache.lucene.codecs.StoredFieldsReader;
+import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.codecs.compressing.CompressionMode;
-import org.apache.lucene.index.StoredFieldVisitor;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 
 /**
  * Lucene 4.1 stored fields format.
- *
- * <p><b>Principle</b></p>
- * <p>This {@link StoredFieldsFormat} compresses blocks of 16KB of documents in
- * order to improve the compression ratio compared to document-level
- * compression. It uses the <a href="http://code.google.com/p/lz4/">LZ4</a>
- * compression algorithm, which is fast to compress and very fast to decompress
- * data. Although the compression method that is used focuses more on speed
- * than on compression ratio, it should provide interesting compression ratios
- * for redundant inputs (such as log files, HTML or plain text).</p>
- * <p><b>File formats</b></p>
- * <p>Stored fields are represented by two files:</p>
- * <ol>
- * <li><a name="field_data" id="field_data"></a>
- * <p>A fields data file (extension <tt>.fdt</tt>). This file stores a compact
- * representation of documents in compressed blocks of 16KB or more. When
- * writing a segment, documents are appended to an in-memory <tt>byte[]</tt>
- * buffer. When its size reaches 16KB or more, some metadata about the documents
- * is flushed to disk, immediately followed by a compressed representation of
- * the buffer using the
- * <a href="http://code.google.com/p/lz4/">LZ4</a>
- * <a href="http://fastcompression.blogspot.fr/2011/05/lz4-explained.html">compression format</a>.</p>
- * <p>Here is a more detailed description of the field data file format:</p>
- * <ul>
- * <li>FieldData (.fdt) --&gt; &lt;Header&gt;, PackedIntsVersion, &lt;Chunk&gt;<sup>ChunkCount</sup></li>
- * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
- * <li>ChunkCount is not known in advance and is the number of chunks necessary to store all document of the segment</li>
- * <li>Chunk --&gt; DocBase, ChunkDocs, DocFieldCounts, DocLengths, &lt;CompressedDocs&gt;</li>
- * <li>DocBase --&gt; the ID of the first document of the chunk as a {@link DataOutput#writeVInt VInt}</li>
- * <li>ChunkDocs --&gt; the number of documents in the chunk as a {@link DataOutput#writeVInt VInt}</li>
- * <li>DocFieldCounts --&gt; the number of stored fields of every document in the chunk, encoded as followed:<ul>
- *   <li>if chunkDocs=1, the unique value is encoded as a {@link DataOutput#writeVInt VInt}</li>
- *   <li>else read a {@link DataOutput#writeVInt VInt} (let's call it <tt>bitsRequired</tt>)<ul>
- *     <li>if <tt>bitsRequired</tt> is <tt>0</tt> then all values are equal, and the common value is the following {@link DataOutput#writeVInt VInt}</li>
- *     <li>else <tt>bitsRequired</tt> is the number of bits required to store any value, and values are stored in a {@link PackedInts packed} array where every value is stored on exactly <tt>bitsRequired</tt> bits</li>
- *   </ul></li>
- * </ul></li>
- * <li>DocLengths --&gt; the lengths of all documents in the chunk, encoded with the same method as DocFieldCounts</li>
- * <li>CompressedDocs --&gt; a compressed representation of &lt;Docs&gt; using the LZ4 compression format</li>
- * <li>Docs --&gt; &lt;Doc&gt;<sup>ChunkDocs</sup></li>
- * <li>Doc --&gt; &lt;FieldNumAndType, Value&gt;<sup>DocFieldCount</sup></li>
- * <li>FieldNumAndType --&gt; a {@link DataOutput#writeVLong VLong}, whose 3 last bits are Type and other bits are FieldNum</li>
- * <li>Type --&gt;<ul>
- *   <li>0: Value is String</li>
- *   <li>1: Value is BinaryValue</li>
- *   <li>2: Value is Int</li>
- *   <li>3: Value is Float</li>
- *   <li>4: Value is Long</li>
- *   <li>5: Value is Double</li>
- *   <li>6, 7: unused</li>
- * </ul></li>
- * <li>FieldNum --&gt; an ID of the field</li>
- * <li>Value --&gt; {@link DataOutput#writeString(String) String} | BinaryValue | Int | Float | Long | Double depending on Type</li>
- * <li>BinaryValue --&gt; ValueLength &lt;Byte&gt;<sup>ValueLength</sup></li>
- * </ul>
- * <p>Notes</p>
- * <ul>
- * <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>
- * <li>In case documents are incompressible, CompressedDocs will be less than
- * 0.5% larger than Docs.</li>
- * </ul>
- * </li>
- * <li><a name="field_index" id="field_index"></a>
- * <p>A fields index file (extension <tt>.fdx</tt>).</p>
- * <ul>
- * <li>FieldsIndex (.fdx) --&gt; &lt;Header&gt;, &lt;ChunkIndex&gt;</li>
- * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- * <li>ChunkIndex: See {@link CompressingStoredFieldsIndexWriter}</li>
- * </ul>
- * </li>
- * </ol>
- * <p><b>Known limitations</b></p>
- * <p>This {@link StoredFieldsFormat} does not support individual documents
- * larger than (<tt>2<sup>31</sup> - 2<sup>14</sup></tt>) bytes.</p>
- * @lucene.experimental
  */
-public final class Lucene41StoredFieldsFormat extends CompressingStoredFieldsFormat {
+@Deprecated
+public class Lucene41StoredFieldsFormat extends StoredFieldsFormat {
+  static final String FORMAT_NAME = "Lucene41StoredFields";
+  static final String SEGMENT_SUFFIX = "";
+  static final CompressionMode COMPRESSION_MODE = CompressionMode.FAST;
+  static final int CHUNK_SIZE = 1 << 14;
+
+  @Override
+  public final StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
+    return new Lucene41StoredFieldsReader(directory, si, SEGMENT_SUFFIX, fn, context, FORMAT_NAME, COMPRESSION_MODE);
+  }
 
-  /** Sole constructor. */
-  public Lucene41StoredFieldsFormat() {
-    super("Lucene41StoredFields", CompressionMode.FAST, 1 << 14);
+  @Override
+  public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
+    throw new UnsupportedOperationException("this codec can only be used for reading");
   }
 
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(compressionMode=" + COMPRESSION_MODE + ", chunkSize=" + CHUNK_SIZE + ")";
+  }
 }

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexReader.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexReader.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexReader.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,214 @@
+package org.apache.lucene.codecs.lucene41;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import static org.apache.lucene.util.BitUtil.zigZagDecode;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Random-access reader for {@code Lucene41CompressingStoredFieldsIndexWriter}.
+ * @deprecated only for reading old segments
+ */
+@Deprecated
+public final class Lucene41StoredFieldsIndexReader implements Cloneable, Accountable {
+
+  private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Lucene41StoredFieldsIndexReader.class);
+
+  final int maxDoc;
+  final int[] docBases;
+  final long[] startPointers;
+  final int[] avgChunkDocs;
+  final long[] avgChunkSizes;
+  final PackedInts.Reader[] docBasesDeltas; // delta from the avg
+  final PackedInts.Reader[] startPointersDeltas; // delta from the avg
+
+  // It is the responsibility of the caller to close fieldsIndexIn after this constructor
+  // has been called
+  public Lucene41StoredFieldsIndexReader(IndexInput fieldsIndexIn, SegmentInfo si) throws IOException {
+    maxDoc = si.getDocCount();
+    int[] docBases = new int[16];
+    long[] startPointers = new long[16];
+    int[] avgChunkDocs = new int[16];
+    long[] avgChunkSizes = new long[16];
+    PackedInts.Reader[] docBasesDeltas = new PackedInts.Reader[16];
+    PackedInts.Reader[] startPointersDeltas = new PackedInts.Reader[16];
+
+    final int packedIntsVersion = fieldsIndexIn.readVInt();
+
+    int blockCount = 0;
+
+    for (;;) {
+      final int numChunks = fieldsIndexIn.readVInt();
+      if (numChunks == 0) {
+        break;
+      }
+      if (blockCount == docBases.length) {
+        final int newSize = ArrayUtil.oversize(blockCount + 1, 8);
+        docBases = Arrays.copyOf(docBases, newSize);
+        startPointers = Arrays.copyOf(startPointers, newSize);
+        avgChunkDocs = Arrays.copyOf(avgChunkDocs, newSize);
+        avgChunkSizes = Arrays.copyOf(avgChunkSizes, newSize);
+        docBasesDeltas = Arrays.copyOf(docBasesDeltas, newSize);
+        startPointersDeltas = Arrays.copyOf(startPointersDeltas, newSize);
+      }
+
+      // doc bases
+      docBases[blockCount] = fieldsIndexIn.readVInt();
+      avgChunkDocs[blockCount] = fieldsIndexIn.readVInt();
+      final int bitsPerDocBase = fieldsIndexIn.readVInt();
+      if (bitsPerDocBase > 32) {
+        throw new CorruptIndexException("Corrupted bitsPerDocBase: " + bitsPerDocBase, fieldsIndexIn);
+      }
+      docBasesDeltas[blockCount] = PackedInts.getReaderNoHeader(fieldsIndexIn, PackedInts.Format.PACKED, packedIntsVersion, numChunks, bitsPerDocBase);
+
+      // start pointers
+      startPointers[blockCount] = fieldsIndexIn.readVLong();
+      avgChunkSizes[blockCount] = fieldsIndexIn.readVLong();
+      final int bitsPerStartPointer = fieldsIndexIn.readVInt();
+      if (bitsPerStartPointer > 64) {
+        throw new CorruptIndexException("Corrupted bitsPerStartPointer: " + bitsPerStartPointer, fieldsIndexIn);
+      }
+      startPointersDeltas[blockCount] = PackedInts.getReaderNoHeader(fieldsIndexIn, PackedInts.Format.PACKED, packedIntsVersion, numChunks, bitsPerStartPointer);
+
+      ++blockCount;
+    }
+
+    this.docBases = Arrays.copyOf(docBases, blockCount);
+    this.startPointers = Arrays.copyOf(startPointers, blockCount);
+    this.avgChunkDocs = Arrays.copyOf(avgChunkDocs, blockCount);
+    this.avgChunkSizes = Arrays.copyOf(avgChunkSizes, blockCount);
+    this.docBasesDeltas = Arrays.copyOf(docBasesDeltas, blockCount);
+    this.startPointersDeltas = Arrays.copyOf(startPointersDeltas, blockCount);
+  }
+
+  private int block(int docID) {
+    int lo = 0, hi = docBases.length - 1;
+    while (lo <= hi) {
+      final int mid = (lo + hi) >>> 1;
+      final int midValue = docBases[mid];
+      if (midValue == docID) {
+        return mid;
+      } else if (midValue < docID) {
+        lo = mid + 1;
+      } else {
+        hi = mid - 1;
+      }
+    }
+    return hi;
+  }
+
+  private int relativeDocBase(int block, int relativeChunk) {
+    final int expected = avgChunkDocs[block] * relativeChunk;
+    final long delta = zigZagDecode(docBasesDeltas[block].get(relativeChunk));
+    return expected + (int) delta;
+  }
+
+  private long relativeStartPointer(int block, int relativeChunk) {
+    final long expected = avgChunkSizes[block] * relativeChunk;
+    final long delta = zigZagDecode(startPointersDeltas[block].get(relativeChunk));
+    return expected + delta;
+  }
+
+  private int relativeChunk(int block, int relativeDoc) {
+    int lo = 0, hi = docBasesDeltas[block].size() - 1;
+    while (lo <= hi) {
+      final int mid = (lo + hi) >>> 1;
+      final int midValue = relativeDocBase(block, mid);
+      if (midValue == relativeDoc) {
+        return mid;
+      } else if (midValue < relativeDoc) {
+        lo = mid + 1;
+      } else {
+        hi = mid - 1;
+      }
+    }
+    return hi;
+  }
+
+  public long getStartPointer(int docID) {
+    if (docID < 0 || docID >= maxDoc) {
+      throw new IllegalArgumentException("docID out of range [0-" + maxDoc + "]: " + docID);
+    }
+    final int block = block(docID);
+    final int relativeChunk = relativeChunk(block, docID - docBases[block]);
+    return startPointers[block] + relativeStartPointer(block, relativeChunk);
+  }
+
+  @Override
+  public Lucene41StoredFieldsIndexReader clone() {
+    return this;
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long res = BASE_RAM_BYTES_USED;
+
+    res += RamUsageEstimator.shallowSizeOf(docBasesDeltas);
+    for (PackedInts.Reader r : docBasesDeltas) {
+      res += r.ramBytesUsed();
+    }
+    res += RamUsageEstimator.shallowSizeOf(startPointersDeltas);
+    for (PackedInts.Reader r : startPointersDeltas) {
+      res += r.ramBytesUsed();
+    }
+
+    res += RamUsageEstimator.sizeOf(docBases);
+    res += RamUsageEstimator.sizeOf(startPointers);
+    res += RamUsageEstimator.sizeOf(avgChunkDocs); 
+    res += RamUsageEstimator.sizeOf(avgChunkSizes);
+
+    return res;
+  }
+
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    
+    long docBaseDeltaBytes = RamUsageEstimator.shallowSizeOf(docBasesDeltas);
+    for (PackedInts.Reader r : docBasesDeltas) {
+      docBaseDeltaBytes += r.ramBytesUsed();
+    }
+    resources.add(Accountables.namedAccountable("doc base deltas", docBaseDeltaBytes));
+    
+    long startPointerDeltaBytes = RamUsageEstimator.shallowSizeOf(startPointersDeltas);
+    for (PackedInts.Reader r : startPointersDeltas) {
+      startPointerDeltaBytes += r.ramBytesUsed();
+    }
+    resources.add(Accountables.namedAccountable("start pointer deltas", startPointerDeltaBytes));
+    
+    return resources;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(blocks=" + docBases.length + ")";
+  }
+}

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsReader.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsReader.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsReader.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,417 @@
+package org.apache.lucene.codecs.lucene41;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.StoredFieldsReader;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.compressing.Decompressor;
+import org.apache.lucene.index.CorruptIndexException;
+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.StoredFieldVisitor;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+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.packed.PackedInts;
+
+/**
+ * {@link StoredFieldsReader} impl for {@code Lucene41StoredFieldsFormat}.
+ * @deprecated only for reading old segments
+ */
+@Deprecated
+final class Lucene41StoredFieldsReader extends StoredFieldsReader {
+
+  // Do not reuse the decompression buffer when there is more than 32kb to decompress
+  private static final int BUFFER_REUSE_THRESHOLD = 1 << 15;
+  
+  static final int         STRING = 0x00;
+  static final int       BYTE_ARR = 0x01;
+  static final int    NUMERIC_INT = 0x02;
+  static final int  NUMERIC_FLOAT = 0x03;
+  static final int   NUMERIC_LONG = 0x04;
+  static final int NUMERIC_DOUBLE = 0x05;
+  
+  static final String CODEC_SFX_IDX = "Index";
+  static final String CODEC_SFX_DAT = "Data";
+  
+  static final int TYPE_BITS = PackedInts.bitsRequired(NUMERIC_DOUBLE);
+  static final int TYPE_MASK = (int) PackedInts.maxValue(TYPE_BITS);
+  
+  static final int VERSION_START = 0;
+  static final int VERSION_BIG_CHUNKS = 1;
+  static final int VERSION_CHECKSUM = 2;
+  static final int VERSION_CURRENT = VERSION_CHECKSUM;
+  
+  /** Extension of stored fields file */
+  public static final String FIELDS_EXTENSION = "fdt";
+  
+  /** Extension of stored fields index file */
+  public static final String FIELDS_INDEX_EXTENSION = "fdx";
+
+  private final int version;
+  private final FieldInfos fieldInfos;
+  private final Lucene41StoredFieldsIndexReader indexReader;
+  private final long maxPointer;
+  private final IndexInput fieldsStream;
+  private final int chunkSize;
+  private final int packedIntsVersion;
+  private final CompressionMode compressionMode;
+  private final Decompressor decompressor;
+  private final BytesRef bytes;
+  private final int numDocs;
+  private boolean closed;
+
+  // used by clone
+  private Lucene41StoredFieldsReader(Lucene41StoredFieldsReader reader) {
+    this.version = reader.version;
+    this.fieldInfos = reader.fieldInfos;
+    this.fieldsStream = reader.fieldsStream.clone();
+    this.indexReader = reader.indexReader.clone();
+    this.maxPointer = reader.maxPointer;
+    this.chunkSize = reader.chunkSize;
+    this.packedIntsVersion = reader.packedIntsVersion;
+    this.compressionMode = reader.compressionMode;
+    this.decompressor = reader.decompressor.clone();
+    this.numDocs = reader.numDocs;
+    this.bytes = new BytesRef(reader.bytes.bytes.length);
+    this.closed = false;
+  }
+
+  /** Sole constructor. */
+  public Lucene41StoredFieldsReader(Directory d, SegmentInfo si, String segmentSuffix, FieldInfos fn,
+      IOContext context, String formatName, CompressionMode compressionMode) throws IOException {
+    this.compressionMode = compressionMode;
+    final String segment = si.name;
+    boolean success = false;
+    fieldInfos = fn;
+    numDocs = si.getDocCount();
+    ChecksumIndexInput indexStream = null;
+    try {
+      final String indexStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION);
+      final String fieldsStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION);
+      // Load the index into memory
+      indexStream = d.openChecksumInput(indexStreamFN, context);
+      final String codecNameIdx = formatName + CODEC_SFX_IDX;
+      version = CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
+      assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
+      indexReader = new Lucene41StoredFieldsIndexReader(indexStream, si);
+
+      long maxPointer = -1;
+      
+      if (version >= VERSION_CHECKSUM) {
+        maxPointer = indexStream.readVLong();
+        CodecUtil.checkFooter(indexStream);
+      } else {
+        CodecUtil.checkEOF(indexStream);
+      }
+      indexStream.close();
+      indexStream = null;
+
+      // Open the data file and read metadata
+      fieldsStream = d.openInput(fieldsStreamFN, context);
+      if (version >= VERSION_CHECKSUM) {
+        if (maxPointer + CodecUtil.footerLength() != fieldsStream.length()) {
+          throw new CorruptIndexException("Invalid fieldsStream maxPointer (file truncated?): maxPointer=" + maxPointer + ", length=" + fieldsStream.length(), fieldsStream);
+        }
+      } else {
+        maxPointer = fieldsStream.length();
+      }
+      this.maxPointer = maxPointer;
+      final String codecNameDat = formatName + CODEC_SFX_DAT;
+      final int fieldsVersion = CodecUtil.checkHeader(fieldsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
+      if (version != fieldsVersion) {
+        throw new CorruptIndexException("Version mismatch between stored fields index and data: " + version + " != " + fieldsVersion, fieldsStream);
+      }
+      assert CodecUtil.headerLength(codecNameDat) == fieldsStream.getFilePointer();
+
+      if (version >= VERSION_BIG_CHUNKS) {
+        chunkSize = fieldsStream.readVInt();
+      } else {
+        chunkSize = -1;
+      }
+      packedIntsVersion = fieldsStream.readVInt();
+      decompressor = compressionMode.newDecompressor();
+      this.bytes = new BytesRef();
+      
+      if (version >= VERSION_CHECKSUM) {
+        // 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.
+        CodecUtil.retrieveChecksum(fieldsStream);
+      }
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this, indexStream);
+      }
+    }
+  }
+
+  /**
+   * @throws AlreadyClosedException if this FieldsReader is closed
+   */
+  private void ensureOpen() throws AlreadyClosedException {
+    if (closed) {
+      throw new AlreadyClosedException("this FieldsReader is closed");
+    }
+  }
+
+  /** 
+   * Close the underlying {@link IndexInput}s.
+   */
+  @Override
+  public void close() throws IOException {
+    if (!closed) {
+      IOUtils.close(fieldsStream);
+      closed = true;
+    }
+  }
+
+  private static void readField(DataInput in, StoredFieldVisitor visitor, FieldInfo info, int bits) throws IOException {
+    switch (bits & TYPE_MASK) {
+      case BYTE_ARR:
+        int length = in.readVInt();
+        byte[] data = new byte[length];
+        in.readBytes(data, 0, length);
+        visitor.binaryField(info, data);
+        break;
+      case STRING:
+        length = in.readVInt();
+        data = new byte[length];
+        in.readBytes(data, 0, length);
+        visitor.stringField(info, new String(data, StandardCharsets.UTF_8));
+        break;
+      case NUMERIC_INT:
+        visitor.intField(info, in.readInt());
+        break;
+      case NUMERIC_FLOAT:
+        visitor.floatField(info, Float.intBitsToFloat(in.readInt()));
+        break;
+      case NUMERIC_LONG:
+        visitor.longField(info, in.readLong());
+        break;
+      case NUMERIC_DOUBLE:
+        visitor.doubleField(info, Double.longBitsToDouble(in.readLong()));
+        break;
+      default:
+        throw new AssertionError("Unknown type flag: " + Integer.toHexString(bits));
+    }
+  }
+
+  private static void skipField(DataInput in, int bits) throws IOException {
+    switch (bits & TYPE_MASK) {
+      case BYTE_ARR:
+      case STRING:
+        final int length = in.readVInt();
+        in.skipBytes(length);
+        break;
+      case NUMERIC_INT:
+      case NUMERIC_FLOAT:
+        in.readInt();
+        break;
+      case NUMERIC_LONG:
+      case NUMERIC_DOUBLE:
+        in.readLong();
+        break;
+      default:
+        throw new AssertionError("Unknown type flag: " + Integer.toHexString(bits));
+    }
+  }
+
+  @Override
+  public void visitDocument(int docID, StoredFieldVisitor visitor)
+      throws IOException {
+    fieldsStream.seek(indexReader.getStartPointer(docID));
+
+    final int docBase = fieldsStream.readVInt();
+    final int chunkDocs = fieldsStream.readVInt();
+    if (docID < docBase
+        || docID >= docBase + chunkDocs
+        || docBase + chunkDocs > numDocs) {
+      throw new CorruptIndexException("Corrupted: docID=" + docID
+          + ", docBase=" + docBase + ", chunkDocs=" + chunkDocs
+          + ", numDocs=" + numDocs, fieldsStream);
+    }
+
+    final int numStoredFields, offset, length, totalLength;
+    if (chunkDocs == 1) {
+      numStoredFields = fieldsStream.readVInt();
+      offset = 0;
+      length = fieldsStream.readVInt();
+      totalLength = length;
+    } else {
+      final int bitsPerStoredFields = fieldsStream.readVInt();
+      if (bitsPerStoredFields == 0) {
+        numStoredFields = fieldsStream.readVInt();
+      } else if (bitsPerStoredFields > 31) {
+        throw new CorruptIndexException("bitsPerStoredFields=" + bitsPerStoredFields, fieldsStream);
+      } else {
+        final long filePointer = fieldsStream.getFilePointer();
+        final PackedInts.Reader reader = PackedInts.getDirectReaderNoHeader(fieldsStream, PackedInts.Format.PACKED, packedIntsVersion, chunkDocs, bitsPerStoredFields);
+        numStoredFields = (int) (reader.get(docID - docBase));
+        fieldsStream.seek(filePointer + PackedInts.Format.PACKED.byteCount(packedIntsVersion, chunkDocs, bitsPerStoredFields));
+      }
+
+      final int bitsPerLength = fieldsStream.readVInt();
+      if (bitsPerLength == 0) {
+        length = fieldsStream.readVInt();
+        offset = (docID - docBase) * length;
+        totalLength = chunkDocs * length;
+      } else if (bitsPerStoredFields > 31) {
+        throw new CorruptIndexException("bitsPerLength=" + bitsPerLength, fieldsStream);
+      } else {
+        final PackedInts.ReaderIterator it = PackedInts.getReaderIteratorNoHeader(fieldsStream, PackedInts.Format.PACKED, packedIntsVersion, chunkDocs, bitsPerLength, 1);
+        int off = 0;
+        for (int i = 0; i < docID - docBase; ++i) {
+          off += it.next();
+        }
+        offset = off;
+        length = (int) it.next();
+        off += length;
+        for (int i = docID - docBase + 1; i < chunkDocs; ++i) {
+          off += it.next();
+        }
+        totalLength = off;
+      }
+    }
+
+    if ((length == 0) != (numStoredFields == 0)) {
+      throw new CorruptIndexException("length=" + length + ", numStoredFields=" + numStoredFields, fieldsStream);
+    }
+    if (numStoredFields == 0) {
+      // nothing to do
+      return;
+    }
+
+    final DataInput documentInput;
+    if (version >= VERSION_BIG_CHUNKS && totalLength >= 2 * chunkSize) {
+      assert chunkSize > 0;
+      assert offset < chunkSize;
+
+      decompressor.decompress(fieldsStream, chunkSize, offset, Math.min(length, chunkSize - offset), bytes);
+      documentInput = new DataInput() {
+
+        int decompressed = bytes.length;
+
+        void fillBuffer() throws IOException {
+          assert decompressed <= length;
+          if (decompressed == length) {
+            throw new EOFException();
+          }
+          final int toDecompress = Math.min(length - decompressed, chunkSize);
+          decompressor.decompress(fieldsStream, toDecompress, 0, toDecompress, bytes);
+          decompressed += toDecompress;
+        }
+
+        @Override
+        public byte readByte() throws IOException {
+          if (bytes.length == 0) {
+            fillBuffer();
+          }
+          --bytes.length;
+          return bytes.bytes[bytes.offset++];
+        }
+
+        @Override
+        public void readBytes(byte[] b, int offset, int len) throws IOException {
+          while (len > bytes.length) {
+            System.arraycopy(bytes.bytes, bytes.offset, b, offset, bytes.length);
+            len -= bytes.length;
+            offset += bytes.length;
+            fillBuffer();
+          }
+          System.arraycopy(bytes.bytes, bytes.offset, b, offset, len);
+          bytes.offset += len;
+          bytes.length -= len;
+        }
+
+      };
+    } else {
+      final BytesRef bytes = totalLength <= BUFFER_REUSE_THRESHOLD ? this.bytes : new BytesRef();
+      decompressor.decompress(fieldsStream, totalLength, offset, length, bytes);
+      assert bytes.length == length;
+      documentInput = new ByteArrayDataInput(bytes.bytes, bytes.offset, bytes.length);
+    }
+
+    for (int fieldIDX = 0; fieldIDX < numStoredFields; fieldIDX++) {
+      final long infoAndBits = documentInput.readVLong();
+      final int fieldNumber = (int) (infoAndBits >>> TYPE_BITS);
+      final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
+
+      final int bits = (int) (infoAndBits & TYPE_MASK);
+      assert bits <= NUMERIC_DOUBLE: "bits=" + Integer.toHexString(bits);
+
+      switch(visitor.needsField(fieldInfo)) {
+        case YES:
+          readField(documentInput, visitor, fieldInfo, bits);
+          break;
+        case NO:
+          skipField(documentInput, bits);
+          break;
+        case STOP:
+          return;
+      }
+    }
+  }
+
+  @Override
+  public StoredFieldsReader clone() {
+    ensureOpen();
+    return new Lucene41StoredFieldsReader(this);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return indexReader.ramBytesUsed();
+  }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.singleton(Accountables.namedAccountable("stored field index", indexReader));
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    if (version >= VERSION_CHECKSUM) {
+      CodecUtil.checksumEntireFile(fieldsStream);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(mode=" + compressionMode + ",chunksize=" + chunkSize + ")";
+  }
+}

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java?rev=1628019&r1=1628018&r2=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java Sat Sep 27 22:44:44 2014
@@ -80,12 +80,12 @@ public class Lucene42Codec extends Codec
   }
   
   @Override
-  public final StoredFieldsFormat storedFieldsFormat() {
+  public StoredFieldsFormat storedFieldsFormat() {
     return fieldsFormat;
   }
   
   @Override
-  public final TermVectorsFormat termVectorsFormat() {
+  public TermVectorsFormat termVectorsFormat() {
     return vectorsFormat;
   }
 

Copied: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java (from r1627946, lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java?p2=lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java&p1=lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java&r1=1627946&r2=1628019&rev=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsFormat.java Sat Sep 27 22:44:44 2014
@@ -17,116 +17,41 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
-import org.apache.lucene.codecs.CodecUtil;
+import java.io.IOException;
+
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.compressing.CompressingStoredFieldsIndexWriter;
-import org.apache.lucene.codecs.compressing.CompressingTermVectorsFormat;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.codecs.TermVectorsWriter;
 import org.apache.lucene.codecs.compressing.CompressionMode;
-import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.packed.BlockPackedWriter;
-import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 
 /**
  * Lucene 4.2 {@link TermVectorsFormat term vectors format}.
- * <p>
- * Very similarly to {@link Lucene41StoredFieldsFormat}, this format is based
- * on compressed chunks of data, with document-level granularity so that a
- * document can never span across distinct chunks. Moreover, data is made as
- * compact as possible:<ul>
- * <li>textual data is compressed using the very light,
- * <a href="http://code.google.com/p/lz4/">LZ4</a> compression algorithm,
- * <li>binary data is written using fixed-size blocks of
- * {@link PackedInts packed ints}.
- * </ul>
- * <p>
- * Term vectors are stored using two files<ul>
- * <li>a data file where terms, frequencies, positions, offsets and payloads
- * are stored,
- * <li>an index file, loaded into memory, used to locate specific documents in
- * the data file.
- * </ul>
- * Looking up term vectors for any document requires at most 1 disk seek.
- * <p><b>File formats</b>
- * <ol>
- * <li><a name="vector_data" id="vector_data"></a>
- * <p>A vector data file (extension <tt>.tvd</tt>). This file stores terms,
- * frequencies, positions, offsets and payloads for every document. Upon writing
- * a new segment, it accumulates data into memory until the buffer used to store
- * terms and payloads grows beyond 4KB. Then it flushes all metadata, terms
- * and positions to disk using <a href="http://code.google.com/p/lz4/">LZ4</a>
- * compression for terms and payloads and
- * {@link BlockPackedWriter blocks of packed ints} for positions.</p>
- * <p>Here is a more detailed description of the field data file format:</p>
- * <ul>
- * <li>VectorData (.tvd) --&gt; &lt;Header&gt;, PackedIntsVersion, ChunkSize, &lt;Chunk&gt;<sup>ChunkCount</sup>, Footer</li>
- * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- * <li>PackedIntsVersion --&gt; {@link PackedInts#VERSION_CURRENT} as a {@link DataOutput#writeVInt VInt}</li>
- * <li>ChunkSize is the number of bytes of terms to accumulate before flushing, as a {@link DataOutput#writeVInt VInt}</li>
- * <li>ChunkCount is not known in advance and is the number of chunks necessary to store all document of the segment</li>
- * <li>Chunk --&gt; DocBase, ChunkDocs, &lt; NumFields &gt;, &lt; FieldNums &gt;, &lt; FieldNumOffs &gt;, &lt; Flags &gt;,
- * &lt; NumTerms &gt;, &lt; TermLengths &gt;, &lt; TermFreqs &gt;, &lt; Positions &gt;, &lt; StartOffsets &gt;, &lt; Lengths &gt;,
- * &lt; PayloadLengths &gt;, &lt; TermAndPayloads &gt;</li>
- * <li>DocBase is the ID of the first doc of the chunk as a {@link DataOutput#writeVInt VInt}</li>
- * <li>ChunkDocs is the number of documents in the chunk</li>
- * <li>NumFields --&gt; DocNumFields<sup>ChunkDocs</sup></li>
- * <li>DocNumFields is the number of fields for each doc, written as a {@link DataOutput#writeVInt VInt} if ChunkDocs==1 and as a {@link PackedInts} array otherwise</li>
- * <li>FieldNums --&gt; FieldNumDelta<sup>TotalDistincFields</sup>, a delta-encoded list of the sorted unique field numbers present in the chunk</li>
- * <li>FieldNumOffs --&gt; FieldNumOff<sup>TotalFields</sup>, as a {@link PackedInts} array</li>
- * <li>FieldNumOff is the offset of the field number in FieldNums</li>
- * <li>TotalFields is the total number of fields (sum of the values of NumFields)</li>
- * <li>Flags --&gt; Bit &lt; FieldFlags &gt;</li>
- * <li>Bit  is a single bit which when true means that fields have the same options for every document in the chunk</li>
- * <li>FieldFlags --&gt; if Bit==1: Flag<sup>TotalDistinctFields</sup> else Flag<sup>TotalFields</sup></li>
- * <li>Flag: a 3-bits int where:<ul>
- * <li>the first bit means that the field has positions</li>
- * <li>the second bit means that the field has offsets</li>
- * <li>the third bit means that the field has payloads</li>
- * </ul></li>
- * <li>NumTerms --&gt; FieldNumTerms<sup>TotalFields</sup></li>
- * <li>FieldNumTerms: the number of terms for each field, using {@link BlockPackedWriter blocks of 64 packed ints}</li>
- * <li>TermLengths --&gt; PrefixLength<sup>TotalTerms</sup> SuffixLength<sup>TotalTerms</sup></li>
- * <li>TotalTerms: total number of terms (sum of NumTerms)</li>
- * <li>PrefixLength: 0 for the first term of a field, the common prefix with the previous term otherwise using {@link BlockPackedWriter blocks of 64 packed ints}</li>
- * <li>SuffixLength: length of the term minus PrefixLength for every term using {@link BlockPackedWriter blocks of 64 packed ints}</li>
- * <li>TermFreqs --&gt; TermFreqMinus1<sup>TotalTerms</sup></li>
- * <li>TermFreqMinus1: (frequency - 1) for each term using  {@link BlockPackedWriter blocks of 64 packed ints}</li>
- * <li>Positions --&gt; PositionDelta<sup>TotalPositions</sup></li>
- * <li>TotalPositions is the sum of frequencies of terms of all fields that have positions</li>
- * <li>PositionDelta: the absolute position for the first position of a term, and the difference with the previous positions for following positions using {@link BlockPackedWriter blocks of 64 packed ints}</li>
- * <li>StartOffsets --&gt; (AvgCharsPerTerm<sup>TotalDistinctFields</sup>) StartOffsetDelta<sup>TotalOffsets</sup></li>
- * <li>TotalOffsets is the sum of frequencies of terms of all fields that have offsets</li>
- * <li>AvgCharsPerTerm: average number of chars per term, encoded as a float on 4 bytes. They are not present if no field has both positions and offsets enabled.</li>
- * <li>StartOffsetDelta: (startOffset - previousStartOffset - AvgCharsPerTerm * PositionDelta). previousStartOffset is 0 for the first offset and AvgCharsPerTerm is 0 if the field has no positions using  {@link BlockPackedWriter blocks of 64 packed ints}</li>
- * <li>Lengths --&gt; LengthMinusTermLength<sup>TotalOffsets</sup></li>
- * <li>LengthMinusTermLength: (endOffset - startOffset - termLength) using  {@link BlockPackedWriter blocks of 64 packed ints}</li>
- * <li>PayloadLengths --&gt; PayloadLength<sup>TotalPayloads</sup></li>
- * <li>TotalPayloads is the sum of frequencies of terms of all fields that have payloads</li>
- * <li>PayloadLength is the payload length encoded using  {@link BlockPackedWriter blocks of 64 packed ints}</li>
- * <li>TermAndPayloads --&gt; LZ4-compressed representation of &lt; FieldTermsAndPayLoads &gt;<sup>TotalFields</sup></li>
- * <li>FieldTermsAndPayLoads --&gt; Terms (Payloads)</li>
- * <li>Terms: term bytes</li>
- * <li>Payloads: payload bytes (if the field has payloads)</li>
- * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * </li>
- * <li><a name="vector_index" id="vector_index"></a>
- * <p>An index file (extension <tt>.tvx</tt>).</p>
- * <ul>
- * <li>VectorIndex (.tvx) --&gt; &lt;Header&gt;, &lt;ChunkIndex&gt;, Footer</li>
- * <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- * <li>ChunkIndex: See {@link CompressingStoredFieldsIndexWriter}</li>
- * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * </li>
- * </ol>
- * @lucene.experimental
+ * @deprecated only for reading old segments
  */
-public final class Lucene42TermVectorsFormat extends CompressingTermVectorsFormat {
+@Deprecated
+public class Lucene42TermVectorsFormat extends TermVectorsFormat {
+  // this is actually what 4.2 TVF wrote!
+  static final String FORMAT_NAME = "Lucene41StoredFields";
+  static final String SEGMENT_SUFFIX = "";
+  static final CompressionMode COMPRESSION_MODE = CompressionMode.FAST;
+  static final int CHUNK_SIZE = 1 << 12;
+
+  @Override
+  public final TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
+    return new Lucene42TermVectorsReader(directory, segmentInfo, SEGMENT_SUFFIX, fieldInfos, context, FORMAT_NAME, COMPRESSION_MODE);
+  }
 
-  /** Sole constructor. */
-  public Lucene42TermVectorsFormat() {
-    super("Lucene41StoredFields", "", CompressionMode.FAST, 1 << 12);
+  @Override
+  public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
+    throw new UnsupportedOperationException("this codec can only be used for reading");
   }
 
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(compressionMode=" + COMPRESSION_MODE + ", chunkSize=" + CHUNK_SIZE + ")";
+  }
 }

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsReader.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsReader.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsReader.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,1073 @@
+package org.apache.lucene.codecs.lucene42;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.compressing.Decompressor;
+import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsIndexReader;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+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.SegmentInfo;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+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.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.packed.BlockPackedReaderIterator;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * {@link TermVectorsReader} for {@code Lucene42TermVectorsFormat}.
+ * @deprecated only for reading old segments
+ */
+@Deprecated
+final class Lucene42TermVectorsReader extends TermVectorsReader implements Closeable {
+
+  private final FieldInfos fieldInfos;
+  final Lucene41StoredFieldsIndexReader indexReader;
+  final IndexInput vectorsStream;
+  private final int version;
+  private final int packedIntsVersion;
+  private final CompressionMode compressionMode;
+  private final Decompressor decompressor;
+  private final int chunkSize;
+  private final int numDocs;
+  private boolean closed;
+  private final BlockPackedReaderIterator reader;
+  
+  static final String VECTORS_EXTENSION = "tvd";
+  static final String VECTORS_INDEX_EXTENSION = "tvx";
+
+  static final String CODEC_SFX_IDX = "Index";
+  static final String CODEC_SFX_DAT = "Data";
+
+  static final int VERSION_START = 0;
+  static final int VERSION_CHECKSUM = 1;
+  static final int VERSION_CURRENT = VERSION_CHECKSUM;
+  
+  static final int BLOCK_SIZE = 64;
+
+  static final int POSITIONS = 0x01;
+  static final int   OFFSETS = 0x02;
+  static final int  PAYLOADS = 0x04;
+  static final int FLAGS_BITS = PackedInts.bitsRequired(POSITIONS | OFFSETS | PAYLOADS);
+
+  // used by clone
+  private Lucene42TermVectorsReader(Lucene42TermVectorsReader reader) {
+    this.fieldInfos = reader.fieldInfos;
+    this.vectorsStream = reader.vectorsStream.clone();
+    this.indexReader = reader.indexReader.clone();
+    this.packedIntsVersion = reader.packedIntsVersion;
+    this.compressionMode = reader.compressionMode;
+    this.decompressor = reader.decompressor.clone();
+    this.chunkSize = reader.chunkSize;
+    this.numDocs = reader.numDocs;
+    this.reader = new BlockPackedReaderIterator(vectorsStream, packedIntsVersion, BLOCK_SIZE, 0);
+    this.version = reader.version;
+    this.closed = false;
+  }
+
+  /** Sole constructor. */
+  public Lucene42TermVectorsReader(Directory d, SegmentInfo si, String segmentSuffix, FieldInfos fn,
+      IOContext context, String formatName, CompressionMode compressionMode) throws IOException {
+    this.compressionMode = compressionMode;
+    final String segment = si.name;
+    boolean success = false;
+    fieldInfos = fn;
+    numDocs = si.getDocCount();
+    ChecksumIndexInput indexStream = null;
+    try {
+      // Load the index into memory
+      final String indexStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION);
+      indexStream = d.openChecksumInput(indexStreamFN, context);
+      final String codecNameIdx = formatName + CODEC_SFX_IDX;
+      version = CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
+      assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
+      indexReader = new Lucene41StoredFieldsIndexReader(indexStream, si);
+      
+      if (version >= VERSION_CHECKSUM) {
+        indexStream.readVLong(); // the end of the data file
+        CodecUtil.checkFooter(indexStream);
+      } else {
+        CodecUtil.checkEOF(indexStream);
+      }
+      indexStream.close();
+      indexStream = null;
+
+      // Open the data file and read metadata
+      final String vectorsStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION);
+      vectorsStream = d.openInput(vectorsStreamFN, context);
+      final String codecNameDat = formatName + CODEC_SFX_DAT;
+      int version2 = CodecUtil.checkHeader(vectorsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
+      if (version != version2) {
+        throw new CorruptIndexException("Version mismatch between stored fields index and data: " + version + " != " + version2, vectorsStream);
+      }
+      assert CodecUtil.headerLength(codecNameDat) == vectorsStream.getFilePointer();
+      
+      long pos = vectorsStream.getFilePointer();
+      if (version >= VERSION_CHECKSUM) {
+        // 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.
+        CodecUtil.retrieveChecksum(vectorsStream);
+        vectorsStream.seek(pos);
+      }
+
+      packedIntsVersion = vectorsStream.readVInt();
+      chunkSize = vectorsStream.readVInt();
+      decompressor = compressionMode.newDecompressor();
+      this.reader = new BlockPackedReaderIterator(vectorsStream, packedIntsVersion, BLOCK_SIZE, 0);
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this, indexStream);
+      }
+    }
+  }
+
+  /**
+   * @throws AlreadyClosedException if this TermVectorsReader is closed
+   */
+  private void ensureOpen() throws AlreadyClosedException {
+    if (closed) {
+      throw new AlreadyClosedException("this FieldsReader is closed");
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!closed) {
+      IOUtils.close(vectorsStream);
+      closed = true;
+    }
+  }
+
+  @Override
+  public TermVectorsReader clone() {
+    return new Lucene42TermVectorsReader(this);
+  }
+
+  @Override
+  public Fields get(int doc) throws IOException {
+    ensureOpen();
+
+    // seek to the right place
+    {
+      final long startPointer = indexReader.getStartPointer(doc);
+      vectorsStream.seek(startPointer);
+    }
+
+    // decode
+    // - docBase: first doc ID of the chunk
+    // - chunkDocs: number of docs of the chunk
+    final int docBase = vectorsStream.readVInt();
+    final int chunkDocs = vectorsStream.readVInt();
+    if (doc < docBase || doc >= docBase + chunkDocs || docBase + chunkDocs > numDocs) {
+      throw new CorruptIndexException("docBase=" + docBase + ",chunkDocs=" + chunkDocs + ",doc=" + doc, vectorsStream);
+    }
+
+    final int skip; // number of fields to skip
+    final int numFields; // number of fields of the document we're looking for
+    final int totalFields; // total number of fields of the chunk (sum for all docs)
+    if (chunkDocs == 1) {
+      skip = 0;
+      numFields = totalFields = vectorsStream.readVInt();
+    } else {
+      reader.reset(vectorsStream, chunkDocs);
+      int sum = 0;
+      for (int i = docBase; i < doc; ++i) {
+        sum += reader.next();
+      }
+      skip = sum;
+      numFields = (int) reader.next();
+      sum += numFields;
+      for (int i = doc + 1; i < docBase + chunkDocs; ++i) {
+        sum += reader.next();
+      }
+      totalFields = sum;
+    }
+
+    if (numFields == 0) {
+      // no vectors
+      return null;
+    }
+
+    // read field numbers that have term vectors
+    final int[] fieldNums;
+    {
+      final int token = vectorsStream.readByte() & 0xFF;
+      assert token != 0; // means no term vectors, cannot happen since we checked for numFields == 0
+      final int bitsPerFieldNum = token & 0x1F;
+      int totalDistinctFields = token >>> 5;
+      if (totalDistinctFields == 0x07) {
+        totalDistinctFields += vectorsStream.readVInt();
+      }
+      ++totalDistinctFields;
+      final PackedInts.ReaderIterator it = PackedInts.getReaderIteratorNoHeader(vectorsStream, PackedInts.Format.PACKED, packedIntsVersion, totalDistinctFields, bitsPerFieldNum, 1);
+      fieldNums = new int[totalDistinctFields];
+      for (int i = 0; i < totalDistinctFields; ++i) {
+        fieldNums[i] = (int) it.next();
+      }
+    }
+
+    // read field numbers and flags
+    final int[] fieldNumOffs = new int[numFields];
+    final PackedInts.Reader flags;
+    {
+      final int bitsPerOff = PackedInts.bitsRequired(fieldNums.length - 1);
+      final PackedInts.Reader allFieldNumOffs = PackedInts.getReaderNoHeader(vectorsStream, PackedInts.Format.PACKED, packedIntsVersion, totalFields, bitsPerOff);
+      switch (vectorsStream.readVInt()) {
+        case 0:
+          final PackedInts.Reader fieldFlags = PackedInts.getReaderNoHeader(vectorsStream, PackedInts.Format.PACKED, packedIntsVersion, fieldNums.length, FLAGS_BITS);
+          PackedInts.Mutable f = PackedInts.getMutable(totalFields, FLAGS_BITS, PackedInts.COMPACT);
+          for (int i = 0; i < totalFields; ++i) {
+            final int fieldNumOff = (int) allFieldNumOffs.get(i);
+            assert fieldNumOff >= 0 && fieldNumOff < fieldNums.length;
+            final int fgs = (int) fieldFlags.get(fieldNumOff);
+            f.set(i, fgs);
+          }
+          flags = f;
+          break;
+        case 1:
+          flags = PackedInts.getReaderNoHeader(vectorsStream, PackedInts.Format.PACKED, packedIntsVersion, totalFields, FLAGS_BITS);
+          break;
+        default:
+          throw new AssertionError();
+      }
+      for (int i = 0; i < numFields; ++i) {
+        fieldNumOffs[i] = (int) allFieldNumOffs.get(skip + i);
+      }
+    }
+
+    // number of terms per field for all fields
+    final PackedInts.Reader numTerms;
+    final int totalTerms;
+    {
+      final int bitsRequired = vectorsStream.readVInt();
+      numTerms = PackedInts.getReaderNoHeader(vectorsStream, PackedInts.Format.PACKED, packedIntsVersion, totalFields, bitsRequired);
+      int sum = 0;
+      for (int i = 0; i < totalFields; ++i) {
+        sum += numTerms.get(i);
+      }
+      totalTerms = sum;
+    }
+
+    // term lengths
+    int docOff = 0, docLen = 0, totalLen;
+    final int[] fieldLengths = new int[numFields];
+    final int[][] prefixLengths = new int[numFields][];
+    final int[][] suffixLengths = new int[numFields][];
+    {
+      reader.reset(vectorsStream, totalTerms);
+      // skip
+      int toSkip = 0;
+      for (int i = 0; i < skip; ++i) {
+        toSkip += numTerms.get(i);
+      }
+      reader.skip(toSkip);
+      // read prefix lengths
+      for (int i = 0; i < numFields; ++i) {
+        final int termCount = (int) numTerms.get(skip + i);
+        final int[] fieldPrefixLengths = new int[termCount];
+        prefixLengths[i] = fieldPrefixLengths;
+        for (int j = 0; j < termCount; ) {
+          final LongsRef next = reader.next(termCount - j);
+          for (int k = 0; k < next.length; ++k) {
+            fieldPrefixLengths[j++] = (int) next.longs[next.offset + k];
+          }
+        }
+      }
+      reader.skip(totalTerms - reader.ord());
+
+      reader.reset(vectorsStream, totalTerms);
+      // skip
+      toSkip = 0;
+      for (int i = 0; i < skip; ++i) {
+        for (int j = 0; j < numTerms.get(i); ++j) {
+          docOff += reader.next();
+        }
+      }
+      for (int i = 0; i < numFields; ++i) {
+        final int termCount = (int) numTerms.get(skip + i);
+        final int[] fieldSuffixLengths = new int[termCount];
+        suffixLengths[i] = fieldSuffixLengths;
+        for (int j = 0; j < termCount; ) {
+          final LongsRef next = reader.next(termCount - j);
+          for (int k = 0; k < next.length; ++k) {
+            fieldSuffixLengths[j++] = (int) next.longs[next.offset + k];
+          }
+        }
+        fieldLengths[i] = sum(suffixLengths[i]);
+        docLen += fieldLengths[i];
+      }
+      totalLen = docOff + docLen;
+      for (int i = skip + numFields; i < totalFields; ++i) {
+        for (int j = 0; j < numTerms.get(i); ++j) {
+          totalLen += reader.next();
+        }
+      }
+    }
+
+    // term freqs
+    final int[] termFreqs = new int[totalTerms];
+    {
+      reader.reset(vectorsStream, totalTerms);
+      for (int i = 0; i < totalTerms; ) {
+        final LongsRef next = reader.next(totalTerms - i);
+        for (int k = 0; k < next.length; ++k) {
+          termFreqs[i++] = 1 + (int) next.longs[next.offset + k];
+        }
+      }
+    }
+
+    // total number of positions, offsets and payloads
+    int totalPositions = 0, totalOffsets = 0, totalPayloads = 0;
+    for (int i = 0, termIndex = 0; i < totalFields; ++i) {
+      final int f = (int) flags.get(i);
+      final int termCount = (int) numTerms.get(i);
+      for (int j = 0; j < termCount; ++j) {
+        final int freq = termFreqs[termIndex++];
+        if ((f & POSITIONS) != 0) {
+          totalPositions += freq;
+        }
+        if ((f & OFFSETS) != 0) {
+          totalOffsets += freq;
+        }
+        if ((f & PAYLOADS) != 0) {
+          totalPayloads += freq;
+        }
+      }
+      assert i != totalFields - 1 || termIndex == totalTerms : termIndex + " " + totalTerms;
+    }
+
+    final int[][] positionIndex = positionIndex(skip, numFields, numTerms, termFreqs);
+    final int[][] positions, startOffsets, lengths;
+    if (totalPositions > 0) {
+      positions = readPositions(skip, numFields, flags, numTerms, termFreqs, POSITIONS, totalPositions, positionIndex);
+    } else {
+      positions = new int[numFields][];
+    }
+
+    if (totalOffsets > 0) {
+      // average number of chars per term
+      final float[] charsPerTerm = new float[fieldNums.length];
+      for (int i = 0; i < charsPerTerm.length; ++i) {
+        charsPerTerm[i] = Float.intBitsToFloat(vectorsStream.readInt());
+      }
+      startOffsets = readPositions(skip, numFields, flags, numTerms, termFreqs, OFFSETS, totalOffsets, positionIndex);
+      lengths = readPositions(skip, numFields, flags, numTerms, termFreqs, OFFSETS, totalOffsets, positionIndex);
+
+      for (int i = 0; i < numFields; ++i) {
+        final int[] fStartOffsets = startOffsets[i];
+        final int[] fPositions = positions[i];
+        // patch offsets from positions
+        if (fStartOffsets != null && fPositions != null) {
+          final float fieldCharsPerTerm = charsPerTerm[fieldNumOffs[i]];
+          for (int j = 0; j < startOffsets[i].length; ++j) {
+            fStartOffsets[j] += (int) (fieldCharsPerTerm * fPositions[j]);
+          }
+        }
+        if (fStartOffsets != null) {
+          final int[] fPrefixLengths = prefixLengths[i];
+          final int[] fSuffixLengths = suffixLengths[i];
+          final int[] fLengths = lengths[i];
+          for (int j = 0, end = (int) numTerms.get(skip + i); j < end; ++j) {
+            // delta-decode start offsets and  patch lengths using term lengths
+            final int termLength = fPrefixLengths[j] + fSuffixLengths[j];
+            lengths[i][positionIndex[i][j]] += termLength;
+            for (int k = positionIndex[i][j] + 1; k < positionIndex[i][j + 1]; ++k) {
+              fStartOffsets[k] += fStartOffsets[k - 1];
+              fLengths[k] += termLength;
+            }
+          }
+        }
+      }
+    } else {
+      startOffsets = lengths = new int[numFields][];
+    }
+    if (totalPositions > 0) {
+      // delta-decode positions
+      for (int i = 0; i < numFields; ++i) {
+        final int[] fPositions = positions[i];
+        final int[] fpositionIndex = positionIndex[i];
+        if (fPositions != null) {
+          for (int j = 0, end = (int) numTerms.get(skip + i); j < end; ++j) {
+            // delta-decode start offsets
+            for (int k = fpositionIndex[j] + 1; k < fpositionIndex[j + 1]; ++k) {
+              fPositions[k] += fPositions[k - 1];
+            }
+          }
+        }
+      }
+    }
+
+    // payload lengths
+    final int[][] payloadIndex = new int[numFields][];
+    int totalPayloadLength = 0;
+    int payloadOff = 0;
+    int payloadLen = 0;
+    if (totalPayloads > 0) {
+      reader.reset(vectorsStream, totalPayloads);
+      // skip
+      int termIndex = 0;
+      for (int i = 0; i < skip; ++i) {
+        final int f = (int) flags.get(i);
+        final int termCount = (int) numTerms.get(i);
+        if ((f & PAYLOADS) != 0) {
+          for (int j = 0; j < termCount; ++j) {
+            final int freq = termFreqs[termIndex + j];
+            for (int k = 0; k < freq; ++k) {
+              final int l = (int) reader.next();
+              payloadOff += l;
+            }
+          }
+        }
+        termIndex += termCount;
+      }
+      totalPayloadLength = payloadOff;
+      // read doc payload lengths
+      for (int i = 0; i < numFields; ++i) {
+        final int f = (int) flags.get(skip + i);
+        final int termCount = (int) numTerms.get(skip + i);
+        if ((f & PAYLOADS) != 0) {
+          final int totalFreq = positionIndex[i][termCount];
+          payloadIndex[i] = new int[totalFreq + 1];
+          int posIdx = 0;
+          payloadIndex[i][posIdx] = payloadLen;
+          for (int j = 0; j < termCount; ++j) {
+            final int freq = termFreqs[termIndex + j];
+            for (int k = 0; k < freq; ++k) {
+              final int payloadLength = (int) reader.next();
+              payloadLen += payloadLength;
+              payloadIndex[i][posIdx+1] = payloadLen;
+              ++posIdx;
+            }
+          }
+          assert posIdx == totalFreq;
+        }
+        termIndex += termCount;
+      }
+      totalPayloadLength += payloadLen;
+      for (int i = skip + numFields; i < totalFields; ++i) {
+        final int f = (int) flags.get(i);
+        final int termCount = (int) numTerms.get(i);
+        if ((f & PAYLOADS) != 0) {
+          for (int j = 0; j < termCount; ++j) {
+            final int freq = termFreqs[termIndex + j];
+            for (int k = 0; k < freq; ++k) {
+              totalPayloadLength += reader.next();
+            }
+          }
+        }
+        termIndex += termCount;
+      }
+      assert termIndex == totalTerms : termIndex + " " + totalTerms;
+    }
+
+    // decompress data
+    final BytesRef suffixBytes = new BytesRef();
+    decompressor.decompress(vectorsStream, totalLen + totalPayloadLength, docOff + payloadOff, docLen + payloadLen, suffixBytes);
+    suffixBytes.length = docLen;
+    final BytesRef payloadBytes = new BytesRef(suffixBytes.bytes, suffixBytes.offset + docLen, payloadLen);
+
+    final int[] fieldFlags = new int[numFields];
+    for (int i = 0; i < numFields; ++i) {
+      fieldFlags[i] = (int) flags.get(skip + i);
+    }
+
+    final int[] fieldNumTerms = new int[numFields];
+    for (int i = 0; i < numFields; ++i) {
+      fieldNumTerms[i] = (int) numTerms.get(skip + i);
+    }
+
+    final int[][] fieldTermFreqs = new int[numFields][];
+    {
+      int termIdx = 0;
+      for (int i = 0; i < skip; ++i) {
+        termIdx += numTerms.get(i);
+      }
+      for (int i = 0; i < numFields; ++i) {
+        final int termCount = (int) numTerms.get(skip + i);
+        fieldTermFreqs[i] = new int[termCount];
+        for (int j = 0; j < termCount; ++j) {
+          fieldTermFreqs[i][j] = termFreqs[termIdx++];
+        }
+      }
+    }
+
+    assert sum(fieldLengths) == docLen : sum(fieldLengths) + " != " + docLen;
+
+    return new TVFields(fieldNums, fieldFlags, fieldNumOffs, fieldNumTerms, fieldLengths,
+        prefixLengths, suffixLengths, fieldTermFreqs,
+        positionIndex, positions, startOffsets, lengths,
+        payloadBytes, payloadIndex,
+        suffixBytes);
+  }
+
+  // field -> term index -> position index
+  private int[][] positionIndex(int skip, int numFields, PackedInts.Reader numTerms, int[] termFreqs) {
+    final int[][] positionIndex = new int[numFields][];
+    int termIndex = 0;
+    for (int i = 0; i < skip; ++i) {
+      final int termCount = (int) numTerms.get(i);
+      termIndex += termCount;
+    }
+    for (int i = 0; i < numFields; ++i) {
+      final int termCount = (int) numTerms.get(skip + i);
+      positionIndex[i] = new int[termCount + 1];
+      for (int j = 0; j < termCount; ++j) {
+        final int freq = termFreqs[termIndex+j];
+        positionIndex[i][j + 1] = positionIndex[i][j] + freq;
+      }
+      termIndex += termCount;
+    }
+    return positionIndex;
+  }
+
+  private int[][] readPositions(int skip, int numFields, PackedInts.Reader flags, PackedInts.Reader numTerms, int[] termFreqs, int flag, final int totalPositions, int[][] positionIndex) throws IOException {
+    final int[][] positions = new int[numFields][];
+    reader.reset(vectorsStream, totalPositions);
+    // skip
+    int toSkip = 0;
+    int termIndex = 0;
+    for (int i = 0; i < skip; ++i) {
+      final int f = (int) flags.get(i);
+      final int termCount = (int) numTerms.get(i);
+      if ((f & flag) != 0) {
+        for (int j = 0; j < termCount; ++j) {
+          final int freq = termFreqs[termIndex+j];
+          toSkip += freq;
+        }
+      }
+      termIndex += termCount;
+    }
+    reader.skip(toSkip);
+    // read doc positions
+    for (int i = 0; i < numFields; ++i) {
+      final int f = (int) flags.get(skip + i);
+      final int termCount = (int) numTerms.get(skip + i);
+      if ((f & flag) != 0) {
+        final int totalFreq = positionIndex[i][termCount];
+        final int[] fieldPositions = new int[totalFreq];
+        positions[i] = fieldPositions;
+        for (int j = 0; j < totalFreq; ) {
+          final LongsRef nextPositions = reader.next(totalFreq - j);
+          for (int k = 0; k < nextPositions.length; ++k) {
+            fieldPositions[j++] = (int) nextPositions.longs[nextPositions.offset + k];
+          }
+        }
+      }
+      termIndex += termCount;
+    }
+    reader.skip(totalPositions - reader.ord());
+    return positions;
+  }
+
+  private class TVFields extends Fields {
+
+    private final int[] fieldNums, fieldFlags, fieldNumOffs, numTerms, fieldLengths;
+    private final int[][] prefixLengths, suffixLengths, termFreqs, positionIndex, positions, startOffsets, lengths, payloadIndex;
+    private final BytesRef suffixBytes, payloadBytes;
+
+    public TVFields(int[] fieldNums, int[] fieldFlags, int[] fieldNumOffs, int[] numTerms, int[] fieldLengths,
+        int[][] prefixLengths, int[][] suffixLengths, int[][] termFreqs,
+        int[][] positionIndex, int[][] positions, int[][] startOffsets, int[][] lengths,
+        BytesRef payloadBytes, int[][] payloadIndex,
+        BytesRef suffixBytes) {
+      this.fieldNums = fieldNums;
+      this.fieldFlags = fieldFlags;
+      this.fieldNumOffs = fieldNumOffs;
+      this.numTerms = numTerms;
+      this.fieldLengths = fieldLengths;
+      this.prefixLengths = prefixLengths;
+      this.suffixLengths = suffixLengths;
+      this.termFreqs = termFreqs;
+      this.positionIndex = positionIndex;
+      this.positions = positions;
+      this.startOffsets = startOffsets;
+      this.lengths = lengths;
+      this.payloadBytes = payloadBytes;
+      this.payloadIndex = payloadIndex;
+      this.suffixBytes = suffixBytes;
+    }
+
+    @Override
+    public Iterator<String> iterator() {
+      return new Iterator<String>() {
+        int i = 0;
+        @Override
+        public boolean hasNext() {
+          return i < fieldNumOffs.length;
+        }
+        @Override
+        public String next() {
+          if (!hasNext()) {
+            throw new NoSuchElementException();
+          }
+          final int fieldNum = fieldNums[fieldNumOffs[i++]];
+          return fieldInfos.fieldInfo(fieldNum).name;
+        }
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
+        }
+      };
+    }
+
+    @Override
+    public Terms terms(String field) throws IOException {
+      final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+      if (fieldInfo == null) {
+        return null;
+      }
+      int idx = -1;
+      for (int i = 0; i < fieldNumOffs.length; ++i) {
+        if (fieldNums[fieldNumOffs[i]] == fieldInfo.number) {
+          idx = i;
+          break;
+        }
+      }
+
+      if (idx == -1 || numTerms[idx] == 0) {
+        // no term
+        return null;
+      }
+      int fieldOff = 0, fieldLen = -1;
+      for (int i = 0; i < fieldNumOffs.length; ++i) {
+        if (i < idx) {
+          fieldOff += fieldLengths[i];
+        } else {
+          fieldLen = fieldLengths[i];
+          break;
+        }
+      }
+      assert fieldLen >= 0;
+      return new TVTerms(numTerms[idx], fieldFlags[idx],
+          prefixLengths[idx], suffixLengths[idx], termFreqs[idx],
+          positionIndex[idx], positions[idx], startOffsets[idx], lengths[idx],
+          payloadIndex[idx], payloadBytes,
+          new BytesRef(suffixBytes.bytes, suffixBytes.offset + fieldOff, fieldLen));
+    }
+
+    @Override
+    public int size() {
+      return fieldNumOffs.length;
+    }
+
+  }
+
+  private class TVTerms extends Terms {
+
+    private final int numTerms, flags;
+    private final int[] prefixLengths, suffixLengths, termFreqs, positionIndex, positions, startOffsets, lengths, payloadIndex;
+    private final BytesRef termBytes, payloadBytes;
+
+    TVTerms(int numTerms, int flags, int[] prefixLengths, int[] suffixLengths, int[] termFreqs,
+        int[] positionIndex, int[] positions, int[] startOffsets, int[] lengths,
+        int[] payloadIndex, BytesRef payloadBytes,
+        BytesRef termBytes) {
+      this.numTerms = numTerms;
+      this.flags = flags;
+      this.prefixLengths = prefixLengths;
+      this.suffixLengths = suffixLengths;
+      this.termFreqs = termFreqs;
+      this.positionIndex = positionIndex;
+      this.positions = positions;
+      this.startOffsets = startOffsets;
+      this.lengths = lengths;
+      this.payloadIndex = payloadIndex;
+      this.payloadBytes = payloadBytes;
+      this.termBytes = termBytes;
+    }
+
+    @Override
+    public TermsEnum iterator(TermsEnum reuse) throws IOException {
+      final TVTermsEnum termsEnum;
+      if (reuse != null && reuse instanceof TVTermsEnum) {
+        termsEnum = (TVTermsEnum) reuse;
+      } else {
+        termsEnum = new TVTermsEnum();
+      }
+      termsEnum.reset(numTerms, flags, prefixLengths, suffixLengths, termFreqs, positionIndex, positions, startOffsets, lengths,
+          payloadIndex, payloadBytes,
+          new ByteArrayDataInput(termBytes.bytes, termBytes.offset, termBytes.length));
+      return termsEnum;
+    }
+
+    @Override
+    public long size() throws IOException {
+      return numTerms;
+    }
+
+    @Override
+    public long getSumTotalTermFreq() throws IOException {
+      return -1L;
+    }
+
+    @Override
+    public long getSumDocFreq() throws IOException {
+      return numTerms;
+    }
+
+    @Override
+    public int getDocCount() throws IOException {
+      return 1;
+    }
+
+    @Override
+    public boolean hasFreqs() {
+      return true;
+    }
+
+    @Override
+    public boolean hasOffsets() {
+      return (flags & OFFSETS) != 0;
+    }
+
+    @Override
+    public boolean hasPositions() {
+      return (flags & POSITIONS) != 0;
+    }
+
+    @Override
+    public boolean hasPayloads() {
+      return (flags & PAYLOADS) != 0;
+    }
+
+  }
+
+  private static class TVTermsEnum extends TermsEnum {
+
+    private int numTerms, startPos, ord;
+    private int[] prefixLengths, suffixLengths, termFreqs, positionIndex, positions, startOffsets, lengths, payloadIndex;
+    private ByteArrayDataInput in;
+    private BytesRef payloads;
+    private final BytesRef term;
+
+    private TVTermsEnum() {
+      term = new BytesRef(16);
+    }
+
+    void reset(int numTerms, int flags, int[] prefixLengths, int[] suffixLengths, int[] termFreqs, int[] positionIndex, int[] positions, int[] startOffsets, int[] lengths,
+        int[] payloadIndex, BytesRef payloads, ByteArrayDataInput in) {
+      this.numTerms = numTerms;
+      this.prefixLengths = prefixLengths;
+      this.suffixLengths = suffixLengths;
+      this.termFreqs = termFreqs;
+      this.positionIndex = positionIndex;
+      this.positions = positions;
+      this.startOffsets = startOffsets;
+      this.lengths = lengths;
+      this.payloadIndex = payloadIndex;
+      this.payloads = payloads;
+      this.in = in;
+      startPos = in.getPosition();
+      reset();
+    }
+
+    void reset() {
+      term.length = 0;
+      in.setPosition(startPos);
+      ord = -1;
+    }
+
+    @Override
+    public BytesRef next() throws IOException {
+      if (ord == numTerms - 1) {
+        return null;
+      } else {
+        assert ord < numTerms;
+        ++ord;
+      }
+
+      // read term
+      term.offset = 0;
+      term.length = prefixLengths[ord] + suffixLengths[ord];
+      if (term.length > term.bytes.length) {
+        term.bytes = ArrayUtil.grow(term.bytes, term.length);
+      }
+      in.readBytes(term.bytes, prefixLengths[ord], suffixLengths[ord]);
+
+      return term;
+    }
+
+    @Override
+    public SeekStatus seekCeil(BytesRef text)
+        throws IOException {
+      if (ord < numTerms && ord >= 0) {
+        final int cmp = term().compareTo(text);
+        if (cmp == 0) {
+          return SeekStatus.FOUND;
+        } else if (cmp > 0) {
+          reset();
+        }
+      }
+      // linear scan
+      while (true) {
+        final BytesRef term = next();
+        if (term == null) {
+          return SeekStatus.END;
+        }
+        final int cmp = term.compareTo(text);
+        if (cmp > 0) {
+          return SeekStatus.NOT_FOUND;
+        } else if (cmp == 0) {
+          return SeekStatus.FOUND;
+        }
+      }
+    }
+
+    @Override
+    public void seekExact(long ord) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BytesRef term() throws IOException {
+      return term;
+    }
+
+    @Override
+    public long ord() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int docFreq() throws IOException {
+      return 1;
+    }
+
+    @Override
+    public long totalTermFreq() throws IOException {
+      return termFreqs[ord];
+    }
+
+    @Override
+    public final DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+      final TVDocsEnum docsEnum;
+      if (reuse != null && reuse instanceof TVDocsEnum) {
+        docsEnum = (TVDocsEnum) reuse;
+      } else {
+        docsEnum = new TVDocsEnum();
+      }
+
+      docsEnum.reset(liveDocs, termFreqs[ord], positionIndex[ord], positions, startOffsets, lengths, payloads, payloadIndex);
+      return docsEnum;
+    }
+
+    @Override
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+      if (positions == null && startOffsets == null) {
+        return null;
+      }
+      // TODO: slightly sheisty
+      return (DocsAndPositionsEnum) docs(liveDocs, reuse, flags);
+    }
+
+  }
+
+  private static class TVDocsEnum extends DocsAndPositionsEnum {
+
+    private Bits liveDocs;
+    private int doc = -1;
+    private int termFreq;
+    private int positionIndex;
+    private int[] positions;
+    private int[] startOffsets;
+    private int[] lengths;
+    private final BytesRef payload;
+    private int[] payloadIndex;
+    private int basePayloadOffset;
+    private int i;
+
+    TVDocsEnum() {
+      payload = new BytesRef();
+    }
+
+    public void reset(Bits liveDocs, int freq, int positionIndex, int[] positions,
+        int[] startOffsets, int[] lengths, BytesRef payloads,
+        int[] payloadIndex) {
+      this.liveDocs = liveDocs;
+      this.termFreq = freq;
+      this.positionIndex = positionIndex;
+      this.positions = positions;
+      this.startOffsets = startOffsets;
+      this.lengths = lengths;
+      this.basePayloadOffset = payloads.offset;
+      this.payload.bytes = payloads.bytes;
+      payload.offset = payload.length = 0;
+      this.payloadIndex = payloadIndex;
+
+      doc = i = -1;
+    }
+
+    private void checkDoc() {
+      if (doc == NO_MORE_DOCS) {
+        throw new IllegalStateException("DocsEnum exhausted");
+      } else if (doc == -1) {
+        throw new IllegalStateException("DocsEnum not started");
+      }
+    }
+
+    private void checkPosition() {
+      checkDoc();
+      if (i < 0) {
+        throw new IllegalStateException("Position enum not started");
+      } else if (i >= termFreq) {
+        throw new IllegalStateException("Read past last position");
+      }
+    }
+
+    @Override
+    public int nextPosition() throws IOException {
+      if (doc != 0) {
+        throw new IllegalStateException();
+      } else if (i >= termFreq - 1) {
+        throw new IllegalStateException("Read past last position");
+      }
+
+      ++i;
+
+      if (payloadIndex != null) {
+        payload.offset = basePayloadOffset + payloadIndex[positionIndex + i];
+        payload.length = payloadIndex[positionIndex + i + 1] - payloadIndex[positionIndex + i];
+      }
+
+      if (positions == null) {
+        return -1;
+      } else {
+        return positions[positionIndex + i];
+      }
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      checkPosition();
+      if (startOffsets == null) {
+        return -1;
+      } else {
+        return startOffsets[positionIndex + i];
+      }
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      checkPosition();
+      if (startOffsets == null) {
+        return -1;
+      } else {
+        return startOffsets[positionIndex + i] + lengths[positionIndex + i];
+      }
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      checkPosition();
+      if (payloadIndex == null || payload.length == 0) {
+        return null;
+      } else {
+        return payload;
+      }
+    }
+
+    @Override
+    public int freq() throws IOException {
+      checkDoc();
+      return termFreq;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      if (doc == -1 && (liveDocs == null || liveDocs.get(0))) {
+        return (doc = 0);
+      } else {
+        return (doc = NO_MORE_DOCS);
+      }
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return slowAdvance(target);
+    }
+
+    @Override
+    public long cost() {
+      return 1;
+    }
+  }
+
+  private static int sum(int[] arr) {
+    int sum = 0;
+    for (int el : arr) {
+      sum += el;
+    }
+    return sum;
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return indexReader.ramBytesUsed();
+  }
+  
+  @Override
+  public Iterable<? extends Accountable> getChildResources() {
+    return Collections.singleton(Accountables.namedAccountable("term vector index", indexReader));
+  }
+  
+  @Override
+  public void checkIntegrity() throws IOException {
+    if (version >= VERSION_CHECKSUM) {
+      CodecUtil.checksumEntireFile(vectorsStream);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(mode=" + compressionMode + ",chunksize=" + chunkSize + ")";
+  }
+}

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java?rev=1628019&r1=1628018&r2=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java Sat Sep 27 22:44:44 2014
@@ -83,12 +83,12 @@ public class Lucene45Codec extends Codec
   }
   
   @Override
-  public final StoredFieldsFormat storedFieldsFormat() {
+  public StoredFieldsFormat storedFieldsFormat() {
     return fieldsFormat;
   }
   
   @Override
-  public final TermVectorsFormat termVectorsFormat() {
+  public TermVectorsFormat termVectorsFormat() {
     return vectorsFormat;
   }