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 [2/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/ ...

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

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

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java?rev=1628019&r1=1628018&r2=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java Sat Sep 27 22:44:44 2014
@@ -15,7 +15,6 @@ import org.apache.lucene.codecs.lucene40
 import org.apache.lucene.codecs.lucene40.Lucene40RWNormsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40RWTermVectorsFormat;
-import org.apache.lucene.util.LuceneTestCase;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -39,7 +38,7 @@ import org.apache.lucene.util.LuceneTest
  */
 @SuppressWarnings("deprecation")
 public class Lucene41RWCodec extends Lucene41Codec {
-  private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
+  private final StoredFieldsFormat fieldsFormat = new Lucene41RWStoredFieldsFormat();
   private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat() {
     @Override
     public FieldInfosWriter getFieldInfosWriter() throws IOException {

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWStoredFieldsFormat.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWStoredFieldsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWStoredFieldsFormat.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,33 @@
+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.IOException;
+
+import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/** read-write version of Lucene41StoredsFieldsFormat for testing */
+public class Lucene41RWStoredFieldsFormat extends Lucene41StoredFieldsFormat {
+  @Override
+  public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
+    return new Lucene41StoredFieldsWriter(directory, si, SEGMENT_SUFFIX, context, FORMAT_NAME, COMPRESSION_MODE, CHUNK_SIZE);
+  }
+}

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexWriter.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexWriter.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsIndexWriter.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,169 @@
+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.zigZagEncode;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * writer for lucene 4.x stored fields/vectors index for testing
+ */
+public final class Lucene41StoredFieldsIndexWriter implements Closeable {
+  
+  static final int BLOCK_SIZE = 1024; // number of chunks to serialize at once
+
+  final IndexOutput fieldsIndexOut;
+  int totalDocs;
+  int blockDocs;
+  int blockChunks;
+  long firstStartPointer;
+  long maxStartPointer;
+  final int[] docBaseDeltas;
+  final long[] startPointerDeltas;
+
+  public Lucene41StoredFieldsIndexWriter(IndexOutput indexOutput) throws IOException {
+    this.fieldsIndexOut = indexOutput;
+    reset();
+    totalDocs = 0;
+    docBaseDeltas = new int[BLOCK_SIZE];
+    startPointerDeltas = new long[BLOCK_SIZE];
+    fieldsIndexOut.writeVInt(PackedInts.VERSION_CURRENT);
+  }
+
+  private void reset() {
+    blockChunks = 0;
+    blockDocs = 0;
+    firstStartPointer = -1; // means unset
+  }
+
+  private void writeBlock() throws IOException {
+    assert blockChunks > 0;
+    fieldsIndexOut.writeVInt(blockChunks);
+
+    // The trick here is that we only store the difference from the average start
+    // pointer or doc base, this helps save bits per value.
+    // And in order to prevent a few chunks that would be far from the average to
+    // raise the number of bits per value for all of them, we only encode blocks
+    // of 1024 chunks at once
+    // See LUCENE-4512
+
+    // doc bases
+    final int avgChunkDocs;
+    if (blockChunks == 1) {
+      avgChunkDocs = 0;
+    } else {
+      avgChunkDocs = Math.round((float) (blockDocs - docBaseDeltas[blockChunks - 1]) / (blockChunks - 1));
+    }
+    fieldsIndexOut.writeVInt(totalDocs - blockDocs); // docBase
+    fieldsIndexOut.writeVInt(avgChunkDocs);
+    int docBase = 0;
+    long maxDelta = 0;
+    for (int i = 0; i < blockChunks; ++i) {
+      final int delta = docBase - avgChunkDocs * i;
+      maxDelta |= zigZagEncode(delta);
+      docBase += docBaseDeltas[i];
+    }
+
+    final int bitsPerDocBase = PackedInts.bitsRequired(maxDelta);
+    fieldsIndexOut.writeVInt(bitsPerDocBase);
+    PackedInts.Writer writer = PackedInts.getWriterNoHeader(fieldsIndexOut,
+        PackedInts.Format.PACKED, blockChunks, bitsPerDocBase, 1);
+    docBase = 0;
+    for (int i = 0; i < blockChunks; ++i) {
+      final long delta = docBase - avgChunkDocs * i;
+      assert PackedInts.bitsRequired(zigZagEncode(delta)) <= writer.bitsPerValue();
+      writer.add(zigZagEncode(delta));
+      docBase += docBaseDeltas[i];
+    }
+    writer.finish();
+
+    // start pointers
+    fieldsIndexOut.writeVLong(firstStartPointer);
+    final long avgChunkSize;
+    if (blockChunks == 1) {
+      avgChunkSize = 0;
+    } else {
+      avgChunkSize = (maxStartPointer - firstStartPointer) / (blockChunks - 1);
+    }
+    fieldsIndexOut.writeVLong(avgChunkSize);
+    long startPointer = 0;
+    maxDelta = 0;
+    for (int i = 0; i < blockChunks; ++i) {
+      startPointer += startPointerDeltas[i];
+      final long delta = startPointer - avgChunkSize * i;
+      maxDelta |= zigZagEncode(delta);
+    }
+
+    final int bitsPerStartPointer = PackedInts.bitsRequired(maxDelta);
+    fieldsIndexOut.writeVInt(bitsPerStartPointer);
+    writer = PackedInts.getWriterNoHeader(fieldsIndexOut, PackedInts.Format.PACKED,
+        blockChunks, bitsPerStartPointer, 1);
+    startPointer = 0;
+    for (int i = 0; i < blockChunks; ++i) {
+      startPointer += startPointerDeltas[i];
+      final long delta = startPointer - avgChunkSize * i;
+      assert PackedInts.bitsRequired(zigZagEncode(delta)) <= writer.bitsPerValue();
+      writer.add(zigZagEncode(delta));
+    }
+    writer.finish();
+  }
+
+  public void writeIndex(int numDocs, long startPointer) throws IOException {
+    if (blockChunks == BLOCK_SIZE) {
+      writeBlock();
+      reset();
+    }
+
+    if (firstStartPointer == -1) {
+      firstStartPointer = maxStartPointer = startPointer;
+    }
+    assert firstStartPointer > 0 && startPointer >= firstStartPointer;
+
+    docBaseDeltas[blockChunks] = numDocs;
+    startPointerDeltas[blockChunks] = startPointer - maxStartPointer;
+
+    ++blockChunks;
+    blockDocs += numDocs;
+    totalDocs += numDocs;
+    maxStartPointer = startPointer;
+  }
+
+  public void finish(int numDocs, long maxPointer) throws IOException {
+    if (numDocs != totalDocs) {
+      throw new IllegalStateException("Expected " + numDocs + " docs, but got " + totalDocs);
+    }
+    if (blockChunks > 0) {
+      writeBlock();
+    }
+    fieldsIndexOut.writeVInt(0); // end marker
+    fieldsIndexOut.writeVLong(maxPointer);
+    CodecUtil.writeFooter(fieldsIndexOut);
+  }
+
+  @Override
+  public void close() throws IOException {
+    fieldsIndexOut.close();
+  }
+
+}

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,313 @@
+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.codecs.lucene41.Lucene41StoredFieldsReader.BYTE_ARR;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.CODEC_SFX_DAT;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.CODEC_SFX_IDX;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.FIELDS_EXTENSION;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.FIELDS_INDEX_EXTENSION;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.NUMERIC_DOUBLE;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.NUMERIC_FLOAT;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.NUMERIC_INT;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.NUMERIC_LONG;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.STRING;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.TYPE_BITS;
+import static org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsReader.VERSION_CURRENT;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.compressing.Compressor;
+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.StorableField;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.GrowableByteArrayDataOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * writer for Lucene 4.1 stored fields for testing
+ */
+final class Lucene41StoredFieldsWriter extends StoredFieldsWriter {
+
+  // hard limit on the maximum number of documents per chunk
+  static final int MAX_DOCUMENTS_PER_CHUNK = 128;
+
+  private final Directory directory;
+  private final String segment;
+  private final String segmentSuffix;
+  private Lucene41StoredFieldsIndexWriter indexWriter;
+  private IndexOutput fieldsStream;
+
+  private final Compressor compressor;
+  private final int chunkSize;
+
+  private final GrowableByteArrayDataOutput bufferedDocs;
+  private int[] numStoredFields; // number of stored fields
+  private int[] endOffsets; // end offsets in bufferedDocs
+  private int docBase; // doc ID at the beginning of the chunk
+  private int numBufferedDocs; // docBase + numBufferedDocs == current doc ID
+
+  /** Sole constructor. */
+  public Lucene41StoredFieldsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
+      String formatName, CompressionMode compressionMode, int chunkSize) throws IOException {
+    assert directory != null;
+    this.directory = directory;
+    this.segment = si.name;
+    this.segmentSuffix = segmentSuffix;
+    this.compressor = compressionMode.newCompressor();
+    this.chunkSize = chunkSize;
+    this.docBase = 0;
+    this.bufferedDocs = new GrowableByteArrayDataOutput(chunkSize);
+    this.numStoredFields = new int[16];
+    this.endOffsets = new int[16];
+    this.numBufferedDocs = 0;
+
+    boolean success = false;
+    IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION), 
+                                                                     context);
+    try {
+      fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION),
+                                                    context);
+
+      final String codecNameIdx = formatName + CODEC_SFX_IDX;
+      final String codecNameDat = formatName + CODEC_SFX_DAT;
+      CodecUtil.writeHeader(indexStream, codecNameIdx, VERSION_CURRENT);
+      CodecUtil.writeHeader(fieldsStream, codecNameDat, VERSION_CURRENT);
+      assert CodecUtil.headerLength(codecNameDat) == fieldsStream.getFilePointer();
+      assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
+
+      indexWriter = new Lucene41StoredFieldsIndexWriter(indexStream);
+      indexStream = null;
+
+      fieldsStream.writeVInt(chunkSize);
+      fieldsStream.writeVInt(PackedInts.VERSION_CURRENT);
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(indexStream);
+        abort();
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      IOUtils.close(fieldsStream, indexWriter);
+    } finally {
+      fieldsStream = null;
+      indexWriter = null;
+    }
+  }
+
+  private int numStoredFieldsInDoc;
+
+  @Override
+  public void startDocument() throws IOException {
+  }
+
+  @Override
+  public void finishDocument() throws IOException {
+    if (numBufferedDocs == this.numStoredFields.length) {
+      final int newLength = ArrayUtil.oversize(numBufferedDocs + 1, 4);
+      this.numStoredFields = Arrays.copyOf(this.numStoredFields, newLength);
+      endOffsets = Arrays.copyOf(endOffsets, newLength);
+    }
+    this.numStoredFields[numBufferedDocs] = numStoredFieldsInDoc;
+    numStoredFieldsInDoc = 0;
+    endOffsets[numBufferedDocs] = bufferedDocs.length;
+    ++numBufferedDocs;
+    if (triggerFlush()) {
+      flush();
+    }
+  }
+
+  private static void saveInts(int[] values, int length, DataOutput out) throws IOException {
+    assert length > 0;
+    if (length == 1) {
+      out.writeVInt(values[0]);
+    } else {
+      boolean allEqual = true;
+      for (int i = 1; i < length; ++i) {
+        if (values[i] != values[0]) {
+          allEqual = false;
+          break;
+        }
+      }
+      if (allEqual) {
+        out.writeVInt(0);
+        out.writeVInt(values[0]);
+      } else {
+        long max = 0;
+        for (int i = 0; i < length; ++i) {
+          max |= values[i];
+        }
+        final int bitsRequired = PackedInts.bitsRequired(max);
+        out.writeVInt(bitsRequired);
+        final PackedInts.Writer w = PackedInts.getWriterNoHeader(out, PackedInts.Format.PACKED, length, bitsRequired, 1);
+        for (int i = 0; i < length; ++i) {
+          w.add(values[i]);
+        }
+        w.finish();
+      }
+    }
+  }
+
+  private void writeHeader(int docBase, int numBufferedDocs, int[] numStoredFields, int[] lengths) throws IOException {
+    // save docBase and numBufferedDocs
+    fieldsStream.writeVInt(docBase);
+    fieldsStream.writeVInt(numBufferedDocs);
+
+    // save numStoredFields
+    saveInts(numStoredFields, numBufferedDocs, fieldsStream);
+
+    // save lengths
+    saveInts(lengths, numBufferedDocs, fieldsStream);
+  }
+
+  private boolean triggerFlush() {
+    return bufferedDocs.length >= chunkSize || // chunks of at least chunkSize bytes
+        numBufferedDocs >= MAX_DOCUMENTS_PER_CHUNK;
+  }
+
+  private void flush() throws IOException {
+    indexWriter.writeIndex(numBufferedDocs, fieldsStream.getFilePointer());
+
+    // transform end offsets into lengths
+    final int[] lengths = endOffsets;
+    for (int i = numBufferedDocs - 1; i > 0; --i) {
+      lengths[i] = endOffsets[i] - endOffsets[i - 1];
+      assert lengths[i] >= 0;
+    }
+    writeHeader(docBase, numBufferedDocs, numStoredFields, lengths);
+
+    // compress stored fields to fieldsStream
+    if (bufferedDocs.length >= 2 * chunkSize) {
+      // big chunk, slice it
+      for (int compressed = 0; compressed < bufferedDocs.length; compressed += chunkSize) {
+        compressor.compress(bufferedDocs.bytes, compressed, Math.min(chunkSize, bufferedDocs.length - compressed), fieldsStream);
+      }
+    } else {
+      compressor.compress(bufferedDocs.bytes, 0, bufferedDocs.length, fieldsStream);
+    }
+
+    // reset
+    docBase += numBufferedDocs;
+    numBufferedDocs = 0;
+    bufferedDocs.length = 0;
+  }
+
+  @Override
+  public void writeField(FieldInfo info, StorableField field)
+      throws IOException {
+
+    ++numStoredFieldsInDoc;
+
+    int bits = 0;
+    final BytesRef bytes;
+    final String string;
+
+    Number number = field.numericValue();
+    if (number != null) {
+      if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
+        bits = NUMERIC_INT;
+      } else if (number instanceof Long) {
+        bits = NUMERIC_LONG;
+      } else if (number instanceof Float) {
+        bits = NUMERIC_FLOAT;
+      } else if (number instanceof Double) {
+        bits = NUMERIC_DOUBLE;
+      } else {
+        throw new IllegalArgumentException("cannot store numeric type " + number.getClass());
+      }
+      string = null;
+      bytes = null;
+    } else {
+      bytes = field.binaryValue();
+      if (bytes != null) {
+        bits = BYTE_ARR;
+        string = null;
+      } else {
+        bits = STRING;
+        string = field.stringValue();
+        if (string == null) {
+          throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
+        }
+      }
+    }
+
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | bits;
+    bufferedDocs.writeVLong(infoAndBits);
+
+    if (bytes != null) {
+      bufferedDocs.writeVInt(bytes.length);
+      bufferedDocs.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+    } else if (string != null) {
+      bufferedDocs.writeString(field.stringValue());
+    } else {
+      if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
+        bufferedDocs.writeInt(number.intValue());
+      } else if (number instanceof Long) {
+        bufferedDocs.writeLong(number.longValue());
+      } else if (number instanceof Float) {
+        bufferedDocs.writeInt(Float.floatToIntBits(number.floatValue()));
+      } else if (number instanceof Double) {
+        bufferedDocs.writeLong(Double.doubleToLongBits(number.doubleValue()));
+      } else {
+        throw new AssertionError("Cannot get here");
+      }
+    }
+  }
+
+  @Override
+  public void abort() {
+    IOUtils.closeWhileHandlingException(this);
+    IOUtils.deleteFilesIgnoringExceptions(directory,
+        IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_EXTENSION),
+        IndexFileNames.segmentFileName(segment, segmentSuffix, FIELDS_INDEX_EXTENSION));
+  }
+
+  @Override
+  public void finish(FieldInfos fis, int numDocs) throws IOException {
+    if (numBufferedDocs > 0) {
+      flush();
+    } else {
+      assert bufferedDocs.length == 0;
+    }
+    if (docBase != numDocs) {
+      throw new RuntimeException("Wrote " + docBase + " docs, finish called with numDocs=" + numDocs);
+    }
+    indexWriter.finish(numDocs, fieldsStream.getFilePointer());
+    CodecUtil.writeFooter(fieldsStream);
+    assert bufferedDocs.length == 0;
+  }
+}

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41StoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41StoredFieldsFormat.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41StoredFieldsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/TestLucene41StoredFieldsFormat.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,28 @@
+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 org.apache.lucene.codecs.Codec;
+import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
+
+public class TestLucene41StoredFieldsFormat extends BaseStoredFieldsFormatTestCase {
+  @Override
+  protected Codec getCodec() {
+    return new Lucene41RWCodec();
+  }
+}

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java?rev=1628019&r1=1628018&r2=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java Sat Sep 27 22:44:44 2014
@@ -24,7 +24,10 @@ import org.apache.lucene.codecs.FieldInf
 import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
 import org.apache.lucene.util.LuceneTestCase;
 
 /**
@@ -35,6 +38,7 @@ public class Lucene42RWCodec extends Luc
 
   private static final DocValuesFormat dv = new Lucene42RWDocValuesFormat();
   private static final NormsFormat norms = new Lucene42RWNormsFormat();
+  private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
 
   private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat() {
     @Override
@@ -64,4 +68,16 @@ public class Lucene42RWCodec extends Luc
   public SegmentInfoFormat segmentInfoFormat() {
     return segmentInfos;
   }
+
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    return storedFields;
+  }
+  
+  private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
+
+  @Override
+  public TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
 }

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWTermVectorsFormat.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWTermVectorsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWTermVectorsFormat.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,34 @@
+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.IOException;
+
+import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/** read-write version of lucene 4.2 term vectors for testing */
+public class Lucene42RWTermVectorsFormat extends Lucene42TermVectorsFormat {
+
+  @Override
+  public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
+    return new Lucene42TermVectorsWriter(directory, segmentInfo, SEGMENT_SUFFIX, context, FORMAT_NAME, COMPRESSION_MODE, CHUNK_SIZE);
+  }
+}

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsWriter.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsWriter.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42TermVectorsWriter.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,712 @@
+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 static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.BLOCK_SIZE;
+import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.CODEC_SFX_DAT;
+import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.CODEC_SFX_IDX;
+import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.FLAGS_BITS;
+import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.OFFSETS;
+import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.PAYLOADS;
+import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.POSITIONS;
+import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.VECTORS_EXTENSION;
+import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.VECTORS_INDEX_EXTENSION;
+import static org.apache.lucene.codecs.lucene42.Lucene42TermVectorsReader.VERSION_CURRENT;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.compressing.Compressor;
+import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsIndexWriter;
+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.store.DataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.GrowableByteArrayDataOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.packed.BlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * writer for Lucene 4.2 term vectors for testing
+ */
+final class Lucene42TermVectorsWriter extends TermVectorsWriter {
+
+  // hard limit on the maximum number of documents per chunk
+  static final int MAX_DOCUMENTS_PER_CHUNK = 128;
+
+  private final Directory directory;
+  private final String segment;
+  private final String segmentSuffix;
+  private Lucene41StoredFieldsIndexWriter indexWriter;
+  private IndexOutput vectorsStream;
+
+  private final Compressor compressor;
+  private final int chunkSize;
+
+  /** a pending doc */
+  private class DocData {
+    final int numFields;
+    final Deque<FieldData> fields;
+    final int posStart, offStart, payStart;
+    DocData(int numFields, int posStart, int offStart, int payStart) {
+      this.numFields = numFields;
+      this.fields = new ArrayDeque<>(numFields);
+      this.posStart = posStart;
+      this.offStart = offStart;
+      this.payStart = payStart;
+    }
+    FieldData addField(int fieldNum, int numTerms, boolean positions, boolean offsets, boolean payloads) {
+      final FieldData field;
+      if (fields.isEmpty()) {
+        field = new FieldData(fieldNum, numTerms, positions, offsets, payloads, posStart, offStart, payStart);
+      } else {
+        final FieldData last = fields.getLast();
+        final int posStart = last.posStart + (last.hasPositions ? last.totalPositions : 0);
+        final int offStart = last.offStart + (last.hasOffsets ? last.totalPositions : 0);
+        final int payStart = last.payStart + (last.hasPayloads ? last.totalPositions : 0);
+        field = new FieldData(fieldNum, numTerms, positions, offsets, payloads, posStart, offStart, payStart);
+      }
+      fields.add(field);
+      return field;
+    }
+  }
+
+  private DocData addDocData(int numVectorFields) {
+    FieldData last = null;
+    for (Iterator<DocData> it = pendingDocs.descendingIterator(); it.hasNext(); ) {
+      final DocData doc = it.next();
+      if (!doc.fields.isEmpty()) {
+        last = doc.fields.getLast();
+        break;
+      }
+    }
+    final DocData doc;
+    if (last == null) {
+      doc = new DocData(numVectorFields, 0, 0, 0);
+    } else {
+      final int posStart = last.posStart + (last.hasPositions ? last.totalPositions : 0);
+      final int offStart = last.offStart + (last.hasOffsets ? last.totalPositions : 0);
+      final int payStart = last.payStart + (last.hasPayloads ? last.totalPositions : 0);
+      doc = new DocData(numVectorFields, posStart, offStart, payStart);
+    }
+    pendingDocs.add(doc);
+    return doc;
+  }
+
+  /** a pending field */
+  private class FieldData {
+    final boolean hasPositions, hasOffsets, hasPayloads;
+    final int fieldNum, flags, numTerms;
+    final int[] freqs, prefixLengths, suffixLengths;
+    final int posStart, offStart, payStart;
+    int totalPositions;
+    int ord;
+    FieldData(int fieldNum, int numTerms, boolean positions, boolean offsets, boolean payloads,
+        int posStart, int offStart, int payStart) {
+      this.fieldNum = fieldNum;
+      this.numTerms = numTerms;
+      this.hasPositions = positions;
+      this.hasOffsets = offsets;
+      this.hasPayloads = payloads;
+      this.flags = (positions ? POSITIONS : 0) | (offsets ? OFFSETS : 0) | (payloads ? PAYLOADS : 0);
+      this.freqs = new int[numTerms];
+      this.prefixLengths = new int[numTerms];
+      this.suffixLengths = new int[numTerms];
+      this.posStart = posStart;
+      this.offStart = offStart;
+      this.payStart = payStart;
+      totalPositions = 0;
+      ord = 0;
+    }
+    void addTerm(int freq, int prefixLength, int suffixLength) {
+      freqs[ord] = freq;
+      prefixLengths[ord] = prefixLength;
+      suffixLengths[ord] = suffixLength;
+      ++ord;
+    }
+    void addPosition(int position, int startOffset, int length, int payloadLength) {
+      if (hasPositions) {
+        if (posStart + totalPositions == positionsBuf.length) {
+          positionsBuf = ArrayUtil.grow(positionsBuf);
+        }
+        positionsBuf[posStart + totalPositions] = position;
+      }
+      if (hasOffsets) {
+        if (offStart + totalPositions == startOffsetsBuf.length) {
+          final int newLength = ArrayUtil.oversize(offStart + totalPositions, 4);
+          startOffsetsBuf = Arrays.copyOf(startOffsetsBuf, newLength);
+          lengthsBuf = Arrays.copyOf(lengthsBuf, newLength);
+        }
+        startOffsetsBuf[offStart + totalPositions] = startOffset;
+        lengthsBuf[offStart + totalPositions] = length;
+      }
+      if (hasPayloads) {
+        if (payStart + totalPositions == payloadLengthsBuf.length) {
+          payloadLengthsBuf = ArrayUtil.grow(payloadLengthsBuf);
+        }
+        payloadLengthsBuf[payStart + totalPositions] = payloadLength;
+      }
+      ++totalPositions;
+    }
+  }
+
+  private int numDocs; // total number of docs seen
+  private final Deque<DocData> pendingDocs; // pending docs
+  private DocData curDoc; // current document
+  private FieldData curField; // current field
+  private final BytesRef lastTerm;
+  private int[] positionsBuf, startOffsetsBuf, lengthsBuf, payloadLengthsBuf;
+  private final GrowableByteArrayDataOutput termSuffixes; // buffered term suffixes
+  private final GrowableByteArrayDataOutput payloadBytes; // buffered term payloads
+  private final BlockPackedWriter writer;
+
+  /** Sole constructor. */
+  public Lucene42TermVectorsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
+      String formatName, CompressionMode compressionMode, int chunkSize) throws IOException {
+    assert directory != null;
+    this.directory = directory;
+    this.segment = si.name;
+    this.segmentSuffix = segmentSuffix;
+    this.compressor = compressionMode.newCompressor();
+    this.chunkSize = chunkSize;
+
+    numDocs = 0;
+    pendingDocs = new ArrayDeque<>();
+    termSuffixes = new GrowableByteArrayDataOutput(ArrayUtil.oversize(chunkSize, 1));
+    payloadBytes = new GrowableByteArrayDataOutput(ArrayUtil.oversize(1, 1));
+    lastTerm = new BytesRef(ArrayUtil.oversize(30, 1));
+
+    boolean success = false;
+    IndexOutput indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION), 
+                                                                     context);
+    try {
+      vectorsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION),
+                                                     context);
+
+      final String codecNameIdx = formatName + CODEC_SFX_IDX;
+      final String codecNameDat = formatName + CODEC_SFX_DAT;
+      CodecUtil.writeHeader(indexStream, codecNameIdx, VERSION_CURRENT);
+      CodecUtil.writeHeader(vectorsStream, codecNameDat, VERSION_CURRENT);
+      assert CodecUtil.headerLength(codecNameDat) == vectorsStream.getFilePointer();
+      assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
+
+      indexWriter = new Lucene41StoredFieldsIndexWriter(indexStream);
+      indexStream = null;
+
+      vectorsStream.writeVInt(PackedInts.VERSION_CURRENT);
+      vectorsStream.writeVInt(chunkSize);
+      writer = new BlockPackedWriter(vectorsStream, BLOCK_SIZE);
+
+      positionsBuf = new int[1024];
+      startOffsetsBuf = new int[1024];
+      lengthsBuf = new int[1024];
+      payloadLengthsBuf = new int[1024];
+
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(indexStream);
+        abort();
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      IOUtils.close(vectorsStream, indexWriter);
+    } finally {
+      vectorsStream = null;
+      indexWriter = null;
+    }
+  }
+
+  @Override
+  public void abort() {
+    IOUtils.closeWhileHandlingException(this);
+    IOUtils.deleteFilesIgnoringExceptions(directory,
+        IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION),
+        IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION));
+  }
+
+  @Override
+  public void startDocument(int numVectorFields) throws IOException {
+    curDoc = addDocData(numVectorFields);
+  }
+
+  @Override
+  public void finishDocument() throws IOException {
+    // append the payload bytes of the doc after its terms
+    termSuffixes.writeBytes(payloadBytes.bytes, payloadBytes.length);
+    payloadBytes.length = 0;
+    ++numDocs;
+    if (triggerFlush()) {
+      flush();
+    }
+    curDoc = null;
+  }
+
+  @Override
+  public void startField(FieldInfo info, int numTerms, boolean positions,
+      boolean offsets, boolean payloads) throws IOException {
+    curField = curDoc.addField(info.number, numTerms, positions, offsets, payloads);
+    lastTerm.length = 0;
+  }
+
+  @Override
+  public void finishField() throws IOException {
+    curField = null;
+  }
+
+  @Override
+  public void startTerm(BytesRef term, int freq) throws IOException {
+    assert freq >= 1;
+    final int prefix = StringHelper.bytesDifference(lastTerm, term);
+    curField.addTerm(freq, prefix, term.length - prefix);
+    termSuffixes.writeBytes(term.bytes, term.offset + prefix, term.length - prefix);
+    // copy last term
+    if (lastTerm.bytes.length < term.length) {
+      lastTerm.bytes = new byte[ArrayUtil.oversize(term.length, 1)];
+    }
+    lastTerm.offset = 0;
+    lastTerm.length = term.length;
+    System.arraycopy(term.bytes, term.offset, lastTerm.bytes, 0, term.length);
+  }
+
+  @Override
+  public void addPosition(int position, int startOffset, int endOffset,
+      BytesRef payload) throws IOException {
+    assert curField.flags != 0;
+    curField.addPosition(position, startOffset, endOffset - startOffset, payload == null ? 0 : payload.length);
+    if (curField.hasPayloads && payload != null) {
+      payloadBytes.writeBytes(payload.bytes, payload.offset, payload.length);
+    }
+  }
+
+  private boolean triggerFlush() {
+    return termSuffixes.length >= chunkSize
+        || pendingDocs.size() >= MAX_DOCUMENTS_PER_CHUNK;
+  }
+
+  private void flush() throws IOException {
+    final int chunkDocs = pendingDocs.size();
+    assert chunkDocs > 0 : chunkDocs;
+
+    // write the index file
+    indexWriter.writeIndex(chunkDocs, vectorsStream.getFilePointer());
+
+    final int docBase = numDocs - chunkDocs;
+    vectorsStream.writeVInt(docBase);
+    vectorsStream.writeVInt(chunkDocs);
+
+    // total number of fields of the chunk
+    final int totalFields = flushNumFields(chunkDocs);
+
+    if (totalFields > 0) {
+      // unique field numbers (sorted)
+      final int[] fieldNums = flushFieldNums();
+      // offsets in the array of unique field numbers
+      flushFields(totalFields, fieldNums);
+      // flags (does the field have positions, offsets, payloads?)
+      flushFlags(totalFields, fieldNums);
+      // number of terms of each field
+      flushNumTerms(totalFields);
+      // prefix and suffix lengths for each field
+      flushTermLengths();
+      // term freqs - 1 (because termFreq is always >=1) for each term
+      flushTermFreqs();
+      // positions for all terms, when enabled
+      flushPositions();
+      // offsets for all terms, when enabled
+      flushOffsets(fieldNums);
+      // payload lengths for all terms, when enabled
+      flushPayloadLengths();
+
+      // compress terms and payloads and write them to the output
+      compressor.compress(termSuffixes.bytes, 0, termSuffixes.length, vectorsStream);
+    }
+
+    // reset
+    pendingDocs.clear();
+    curDoc = null;
+    curField = null;
+    termSuffixes.length = 0;
+  }
+
+  private int flushNumFields(int chunkDocs) throws IOException {
+    if (chunkDocs == 1) {
+      final int numFields = pendingDocs.getFirst().numFields;
+      vectorsStream.writeVInt(numFields);
+      return numFields;
+    } else {
+      writer.reset(vectorsStream);
+      int totalFields = 0;
+      for (DocData dd : pendingDocs) {
+        writer.add(dd.numFields);
+        totalFields += dd.numFields;
+      }
+      writer.finish();
+      return totalFields;
+    }
+  }
+
+  /** Returns a sorted array containing unique field numbers */
+  private int[] flushFieldNums() throws IOException {
+    SortedSet<Integer> fieldNums = new TreeSet<>();
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        fieldNums.add(fd.fieldNum);
+      }
+    }
+
+    final int numDistinctFields = fieldNums.size();
+    assert numDistinctFields > 0;
+    final int bitsRequired = PackedInts.bitsRequired(fieldNums.last());
+    final int token = (Math.min(numDistinctFields - 1, 0x07) << 5) | bitsRequired;
+    vectorsStream.writeByte((byte) token);
+    if (numDistinctFields - 1 >= 0x07) {
+      vectorsStream.writeVInt(numDistinctFields - 1 - 0x07);
+    }
+    final PackedInts.Writer writer = PackedInts.getWriterNoHeader(vectorsStream, PackedInts.Format.PACKED, fieldNums.size(), bitsRequired, 1);
+    for (Integer fieldNum : fieldNums) {
+      writer.add(fieldNum);
+    }
+    writer.finish();
+
+    int[] fns = new int[fieldNums.size()];
+    int i = 0;
+    for (Integer key : fieldNums) {
+      fns[i++] = key;
+    }
+    return fns;
+  }
+
+  private void flushFields(int totalFields, int[] fieldNums) throws IOException {
+    final PackedInts.Writer writer = PackedInts.getWriterNoHeader(vectorsStream, PackedInts.Format.PACKED, totalFields, PackedInts.bitsRequired(fieldNums.length - 1), 1);
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        final int fieldNumIndex = Arrays.binarySearch(fieldNums, fd.fieldNum);
+        assert fieldNumIndex >= 0;
+        writer.add(fieldNumIndex);
+      }
+    }
+    writer.finish();
+  }
+
+  private void flushFlags(int totalFields, int[] fieldNums) throws IOException {
+    // check if fields always have the same flags
+    boolean nonChangingFlags = true;
+    int[] fieldFlags = new int[fieldNums.length];
+    Arrays.fill(fieldFlags, -1);
+    outer:
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        final int fieldNumOff = Arrays.binarySearch(fieldNums, fd.fieldNum);
+        assert fieldNumOff >= 0;
+        if (fieldFlags[fieldNumOff] == -1) {
+          fieldFlags[fieldNumOff] = fd.flags;
+        } else if (fieldFlags[fieldNumOff] != fd.flags) {
+          nonChangingFlags = false;
+          break outer;
+        }
+      }
+    }
+
+    if (nonChangingFlags) {
+      // write one flag per field num
+      vectorsStream.writeVInt(0);
+      final PackedInts.Writer writer = PackedInts.getWriterNoHeader(vectorsStream, PackedInts.Format.PACKED, fieldFlags.length, FLAGS_BITS, 1);
+      for (int flags : fieldFlags) {
+        assert flags >= 0;
+        writer.add(flags);
+      }
+      assert writer.ord() == fieldFlags.length - 1;
+      writer.finish();
+    } else {
+      // write one flag for every field instance
+      vectorsStream.writeVInt(1);
+      final PackedInts.Writer writer = PackedInts.getWriterNoHeader(vectorsStream, PackedInts.Format.PACKED, totalFields, FLAGS_BITS, 1);
+      for (DocData dd : pendingDocs) {
+        for (FieldData fd : dd.fields) {
+          writer.add(fd.flags);
+        }
+      }
+      assert writer.ord() == totalFields - 1;
+      writer.finish();
+    }
+  }
+
+  private void flushNumTerms(int totalFields) throws IOException {
+    int maxNumTerms = 0;
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        maxNumTerms |= fd.numTerms;
+      }
+    }
+    final int bitsRequired = PackedInts.bitsRequired(maxNumTerms);
+    vectorsStream.writeVInt(bitsRequired);
+    final PackedInts.Writer writer = PackedInts.getWriterNoHeader(
+        vectorsStream, PackedInts.Format.PACKED, totalFields, bitsRequired, 1);
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        writer.add(fd.numTerms);
+      }
+    }
+    assert writer.ord() == totalFields - 1;
+    writer.finish();
+  }
+
+  private void flushTermLengths() throws IOException {
+    writer.reset(vectorsStream);
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        for (int i = 0; i < fd.numTerms; ++i) {
+          writer.add(fd.prefixLengths[i]);
+        }
+      }
+    }
+    writer.finish();
+    writer.reset(vectorsStream);
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        for (int i = 0; i < fd.numTerms; ++i) {
+          writer.add(fd.suffixLengths[i]);
+        }
+      }
+    }
+    writer.finish();
+  }
+
+  private void flushTermFreqs() throws IOException {
+    writer.reset(vectorsStream);
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        for (int i = 0; i < fd.numTerms; ++i) {
+          writer.add(fd.freqs[i] - 1);
+        }
+      }
+    }
+    writer.finish();
+  }
+
+  private void flushPositions() throws IOException {
+    writer.reset(vectorsStream);
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        if (fd.hasPositions) {
+          int pos = 0;
+          for (int i = 0; i < fd.numTerms; ++i) {
+            int previousPosition = 0;
+            for (int j = 0; j < fd.freqs[i]; ++j) {
+              final int position = positionsBuf[fd .posStart + pos++];
+              writer.add(position - previousPosition);
+              previousPosition = position;
+            }
+          }
+          assert pos == fd.totalPositions;
+        }
+      }
+    }
+    writer.finish();
+  }
+
+  private void flushOffsets(int[] fieldNums) throws IOException {
+    boolean hasOffsets = false;
+    long[] sumPos = new long[fieldNums.length];
+    long[] sumOffsets = new long[fieldNums.length];
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        hasOffsets |= fd.hasOffsets;
+        if (fd.hasOffsets && fd.hasPositions) {
+          final int fieldNumOff = Arrays.binarySearch(fieldNums, fd.fieldNum);
+          int pos = 0;
+          for (int i = 0; i < fd.numTerms; ++i) {
+            int previousPos = 0;
+            int previousOff = 0;
+            for (int j = 0; j < fd.freqs[i]; ++j) {
+              final int position = positionsBuf[fd.posStart + pos];
+              final int startOffset = startOffsetsBuf[fd.offStart + pos];
+              sumPos[fieldNumOff] += position - previousPos;
+              sumOffsets[fieldNumOff] += startOffset - previousOff;
+              previousPos = position;
+              previousOff = startOffset;
+              ++pos;
+            }
+          }
+          assert pos == fd.totalPositions;
+        }
+      }
+    }
+
+    if (!hasOffsets) {
+      // nothing to do
+      return;
+    }
+
+    final float[] charsPerTerm = new float[fieldNums.length];
+    for (int i = 0; i < fieldNums.length; ++i) {
+      charsPerTerm[i] = (sumPos[i] <= 0 || sumOffsets[i] <= 0) ? 0 : (float) ((double) sumOffsets[i] / sumPos[i]);
+    }
+
+    // start offsets
+    for (int i = 0; i < fieldNums.length; ++i) {
+      vectorsStream.writeInt(Float.floatToRawIntBits(charsPerTerm[i]));
+    }
+
+    writer.reset(vectorsStream);
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        if ((fd.flags & OFFSETS) != 0) {
+          final int fieldNumOff = Arrays.binarySearch(fieldNums, fd.fieldNum);
+          final float cpt = charsPerTerm[fieldNumOff];
+          int pos = 0;
+          for (int i = 0; i < fd.numTerms; ++i) {
+            int previousPos = 0;
+            int previousOff = 0;
+            for (int j = 0; j < fd.freqs[i]; ++j) {
+              final int position = fd.hasPositions ? positionsBuf[fd.posStart + pos] : 0;
+              final int startOffset = startOffsetsBuf[fd.offStart + pos];
+              writer.add(startOffset - previousOff - (int) (cpt * (position - previousPos)));
+              previousPos = position;
+              previousOff = startOffset;
+              ++pos;
+            }
+          }
+        }
+      }
+    }
+    writer.finish();
+
+    // lengths
+    writer.reset(vectorsStream);
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        if ((fd.flags & OFFSETS) != 0) {
+          int pos = 0;
+          for (int i = 0; i < fd.numTerms; ++i) {
+            for (int j = 0; j < fd.freqs[i]; ++j) {
+              writer.add(lengthsBuf[fd.offStart + pos++] - fd.prefixLengths[i] - fd.suffixLengths[i]);
+            }
+          }
+          assert pos == fd.totalPositions;
+        }
+      }
+    }
+    writer.finish();
+  }
+
+  private void flushPayloadLengths() throws IOException {
+    writer.reset(vectorsStream);
+    for (DocData dd : pendingDocs) {
+      for (FieldData fd : dd.fields) {
+        if (fd.hasPayloads) {
+          for (int i = 0; i < fd.totalPositions; ++i) {
+            writer.add(payloadLengthsBuf[fd.payStart + i]);
+          }
+        }
+      }
+    }
+    writer.finish();
+  }
+
+  @Override
+  public void finish(FieldInfos fis, int numDocs) throws IOException {
+    if (!pendingDocs.isEmpty()) {
+      flush();
+    }
+    if (numDocs != this.numDocs) {
+      throw new RuntimeException("Wrote " + this.numDocs + " docs, finish called with numDocs=" + numDocs);
+    }
+    indexWriter.finish(numDocs, vectorsStream.getFilePointer());
+    CodecUtil.writeFooter(vectorsStream);
+  }
+
+  @Override
+  public void addProx(int numProx, DataInput positions, DataInput offsets)
+      throws IOException {
+    assert (curField.hasPositions) == (positions != null);
+    assert (curField.hasOffsets) == (offsets != null);
+
+    if (curField.hasPositions) {
+      final int posStart = curField.posStart + curField.totalPositions;
+      if (posStart + numProx > positionsBuf.length) {
+        positionsBuf = ArrayUtil.grow(positionsBuf, posStart + numProx);
+      }
+      int position = 0;
+      if (curField.hasPayloads) {
+        final int payStart = curField.payStart + curField.totalPositions;
+        if (payStart + numProx > payloadLengthsBuf.length) {
+          payloadLengthsBuf = ArrayUtil.grow(payloadLengthsBuf, payStart + numProx);
+        }
+        for (int i = 0; i < numProx; ++i) {
+          final int code = positions.readVInt();
+          if ((code & 1) != 0) {
+            // This position has a payload
+            final int payloadLength = positions.readVInt();
+            payloadLengthsBuf[payStart + i] = payloadLength;
+            payloadBytes.copyBytes(positions, payloadLength);
+          } else {
+            payloadLengthsBuf[payStart + i] = 0;
+          }
+          position += code >>> 1;
+          positionsBuf[posStart + i] = position;
+        }
+      } else {
+        for (int i = 0; i < numProx; ++i) {
+          position += (positions.readVInt() >>> 1);
+          positionsBuf[posStart + i] = position;
+        }
+      }
+    }
+
+    if (curField.hasOffsets) {
+      final int offStart = curField.offStart + curField.totalPositions;
+      if (offStart + numProx > startOffsetsBuf.length) {
+        final int newLength = ArrayUtil.oversize(offStart + numProx, 4);
+        startOffsetsBuf = Arrays.copyOf(startOffsetsBuf, newLength);
+        lengthsBuf = Arrays.copyOf(lengthsBuf, newLength);
+      }
+      int lastOffset = 0, startOffset, endOffset;
+      for (int i = 0; i < numProx; ++i) {
+        startOffset = lastOffset + offsets.readVInt();
+        endOffset = startOffset + offsets.readVInt();
+        lastOffset = endOffset;
+        startOffsetsBuf[offStart + i] = startOffset;
+        lengthsBuf[offStart + i] = endOffset - startOffset;
+      }
+    }
+
+    curField.totalPositions += numProx;
+  }
+}

Added: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/TestLucene42TermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/TestLucene42TermVectorsFormat.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/TestLucene42TermVectorsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/TestLucene42TermVectorsFormat.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,28 @@
+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 org.apache.lucene.codecs.Codec;
+import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
+
+public class TestLucene42TermVectorsFormat extends BaseTermVectorsFormatTestCase {
+  @Override
+  protected Codec getCodec() {
+    return new Lucene42RWCodec();
+  }
+}

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java?rev=1628019&r1=1628018&r2=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene45/Lucene45RWCodec.java Sat Sep 27 22:44:44 2014
@@ -24,10 +24,14 @@ import org.apache.lucene.codecs.FieldInf
 import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40RWSegmentInfoFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosWriter;
 import org.apache.lucene.codecs.lucene42.Lucene42RWNormsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
 
 /**
  * Read-write version of {@link Lucene45Codec} for testing.
@@ -67,4 +71,18 @@ public class Lucene45RWCodec extends Luc
   public SegmentInfoFormat segmentInfoFormat() {
     return segmentInfos;
   }
+  
+  private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
+
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    return storedFields;
+  }
+  
+  private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
+
+  @Override
+  public TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
 }

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWCodec.java?rev=1628019&r1=1628018&r2=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWCodec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene46/Lucene46RWCodec.java Sat Sep 27 22:44:44 2014
@@ -21,7 +21,11 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.SegmentInfoWriter;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42RWNormsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
 import org.apache.lucene.codecs.lucene45.Lucene45RWDocValuesFormat;
 
 /**
@@ -50,4 +54,18 @@ public class Lucene46RWCodec extends Luc
   public SegmentInfoFormat segmentInfoFormat() {
     return segmentInfos;
   }
+  
+  private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
+
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    return storedFields;
+  }
+  
+  private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
+
+  @Override
+  public TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
 }

Modified: lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java?rev=1628019&r1=1628018&r2=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene49/Lucene49RWCodec.java Sat Sep 27 22:44:44 2014
@@ -20,6 +20,10 @@ package org.apache.lucene.codecs.lucene4
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42RWTermVectorsFormat;
 import org.apache.lucene.codecs.lucene46.Lucene46RWSegmentInfoFormat;
 
 /**
@@ -48,4 +52,18 @@ public class Lucene49RWCodec extends Luc
   public SegmentInfoFormat segmentInfoFormat() {
     return segmentInfos;
   }
+  
+  private static final StoredFieldsFormat storedFields = new Lucene41RWStoredFieldsFormat();
+
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    return storedFields;
+  }
+  
+  private final TermVectorsFormat vectorsFormat = new Lucene42RWTermVectorsFormat();
+
+  @Override
+  public TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
 }

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java?rev=1628019&r1=1628018&r2=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50Codec.java Sat Sep 27 22:44:44 2014
@@ -27,8 +27,6 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
 import org.apache.lucene.codecs.lucene49.Lucene49NormsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
@@ -44,8 +42,8 @@ import org.apache.lucene.codecs.perfield
  * @lucene.experimental
  */
 public class Lucene50Codec extends Codec {
-  private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
-  private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
+  private final StoredFieldsFormat fieldsFormat = new Lucene50StoredFieldsFormat();
+  private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
   private final FieldInfosFormat fieldInfosFormat = new Lucene50FieldInfosFormat();
   private final SegmentInfoFormat segmentInfosFormat = new Lucene50SegmentInfoFormat();
   private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();

Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50StoredFieldsFormat.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,124 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * 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 org.apache.lucene.codecs.CodecUtil;
+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.compressing.CompressionMode;
+import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Lucene 5.0 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 Lucene50StoredFieldsFormat extends CompressingStoredFieldsFormat {
+
+  /** Sole constructor. */
+  public Lucene50StoredFieldsFormat() {
+    super("Lucene50StoredFields", CompressionMode.FAST, 1 << 14);
+  }
+
+}

Added: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java?rev=1628019&view=auto
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java (added)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50TermVectorsFormat.java Sat Sep 27 22:44:44 2014
@@ -0,0 +1,131 @@
+package org.apache.lucene.codecs.lucene50;
+
+/*
+ * 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 org.apache.lucene.codecs.CodecUtil;
+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.compressing.CompressionMode;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.packed.BlockPackedWriter;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Lucene 5.0 {@link TermVectorsFormat term vectors format}.
+ * <p>
+ * Very similarly to {@link Lucene50StoredFieldsFormat}, 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
+ */
+public final class Lucene50TermVectorsFormat extends CompressingTermVectorsFormat {
+
+  /** Sole constructor. */
+  public Lucene50TermVectorsFormat() {
+    super("Lucene50TermVectors", "", CompressionMode.FAST, 1 << 12);
+  }
+
+}

Modified: lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html?rev=1628019&r1=1628018&r2=1628019&view=diff
==============================================================================
--- lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html (original)
+++ lucene/dev/branches/lucene5969/lucene/core/src/java/org/apache/lucene/codecs/lucene50/package.html Sat Sep 27 22:44:44 2014
@@ -147,7 +147,7 @@ its numbering.</p>
    This contains the set of field names used in the index.
 </li>
 <li>
-{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Stored Field values}. 
+{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}. 
 This contains, for each document, a list of attribute-value pairs, where the attributes 
 are field names. These are used to store auxiliary information about the document, such as 
 its title, url, or an identifier to access a database. The set of stored fields are what is 
@@ -178,7 +178,7 @@ For each field in each document, a value
 that is multiplied into the score for hits on that field.
 </li>
 <li>
-{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vectors}. 
+{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}. 
 For each field in each document, the term vector (sometimes
 called document vector) may be stored. A term vector consists of term text and
 term frequency. To add Term Vectors to your index see the 
@@ -254,12 +254,12 @@ systems that frequently run out of file 
 <td>Stores information about the fields</td>
 </tr>
 <tr>
-<td>{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Index}</td>
+<td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
 <td>.fdx</td>
 <td>Contains pointers to field data</td>
 </tr>
 <tr>
-<td>{@link org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat Field Data}</td>
+<td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
 <td>.fdt</td>
 <td>The stored fields for documents</td>
 </tr>
@@ -299,17 +299,17 @@ systems that frequently run out of file 
 <td>Encodes additional scoring factors or other per-document information.</td>
 </tr>
 <tr>
-<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Index}</td>
+<td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
 <td>.tvx</td>
 <td>Stores offset into the document data file</td>
 </tr>
 <tr>
-<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Documents}</td>
+<td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Documents}</td>
 <td>.tvd</td>
 <td>Contains information about each document that has term vectors</td>
 </tr>
 <tr>
-<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Fields}</td>
+<td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Fields}</td>
 <td>.tvf</td>
 <td>The field level info about term vectors</td>
 </tr>