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/10/05 16:43:51 UTC

svn commit: r1629501 [3/7] - in /lucene/dev/branches/branch_5x: ./ dev-tools/ lucene/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene40/ lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene41/ lucene/backward-codecs/src/java/...

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java Sun Oct  5 14:43:47 2014
@@ -19,148 +19,15 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.SmallFloat;
-import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.packed.BlockPackedWriter;
-import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
-import org.apache.lucene.util.packed.PackedInts;
 
 /**
  * Lucene 4.5 DocValues format.
- * <p>
- * Encodes the four per-document value types (Numeric,Binary,Sorted,SortedSet) with these strategies:
- * <p>
- * {@link DocValuesType#NUMERIC NUMERIC}:
- * <ul>
- *    <li>Delta-compressed: per-document integers written in blocks of 16k. For each block
- *        the minimum value in that block is encoded, and each entry is a delta from that 
- *        minimum value. Each block of deltas is compressed with bitpacking. For more 
- *        information, see {@link BlockPackedWriter}.
- *    <li>Table-compressed: when the number of unique values is very small (&lt; 256), and
- *        when there are unused "gaps" in the range of values used (such as {@link SmallFloat}), 
- *        a lookup table is written instead. Each per-document entry is instead the ordinal 
- *        to this table, and those ordinals are compressed with bitpacking ({@link PackedInts}). 
- *    <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest
- *        common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
- * </ul>
- * <p>
- * {@link DocValuesType#BINARY BINARY}:
- * <ul>
- *    <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
- *        Each document's value can be addressed directly with multiplication ({@code docID * length}). 
- *    <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses 
- *        for each document. The addresses are written in blocks of 16k, with the current absolute
- *        start for the block, and the average (expected) delta per entry. For each document the 
- *        deviation from the delta (actual - expected) is written.
- *    <li>Prefix-compressed Binary: values are written in chunks of 16, with the first value written
- *        completely and other values sharing prefixes. chunk addresses are written in blocks of 16k,
- *        with the current absolute start for the block, and the average (expected) delta per entry. 
- *        For each chunk the deviation from the delta (actual - expected) is written.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED SORTED}:
- * <ul>
- *    <li>Sorted: a mapping of ordinals to deduplicated terms is written as Prefix-Compressed Binary, 
- *        along with the per-document ordinals written using one of the numeric strategies above.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED_SET SORTED_SET}:
- * <ul>
- *    <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Prefix-Compressed Binary, 
- *        an ordinal list and per-document index into this list are written using the numeric strategies 
- *        above. 
- * </ul>
- * <p>
- * Files:
- * <ol>
- *   <li><tt>.dvd</tt>: DocValues data</li>
- *   <li><tt>.dvm</tt>: DocValues metadata</li>
- * </ol>
- * <ol>
- *   <li><a name="dvm" id="dvm"></a>
- *   <p>The DocValues metadata or .dvm file.</p>
- *   <p>For DocValues field, this stores metadata, such as the offset into the 
- *      DocValues data (.dvd)</p>
- *   <p>DocValues metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>Entry --&gt; NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry</li>
- *     <li>NumericEntry --&gt; GCDNumericEntry | TableNumericEntry | DeltaNumericEntry</li>
- *     <li>GCDNumericEntry --&gt; NumericHeader,MinValue,GCD</li>
- *     <li>TableNumericEntry --&gt; NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup></li>
- *     <li>DeltaNumericEntry --&gt; NumericHeader</li>
- *     <li>NumericHeader --&gt; FieldNumber,EntryType,NumericType,MissingOffset,PackedVersion,DataOffset,Count,BlockSize</li>
- *     <li>BinaryEntry --&gt; FixedBinaryEntry | VariableBinaryEntry | PrefixBinaryEntry</li>
- *     <li>FixedBinaryEntry --&gt; BinaryHeader</li>
- *     <li>VariableBinaryEntry --&gt; BinaryHeader,AddressOffset,PackedVersion,BlockSize</li>
- *     <li>PrefixBinaryEntry --&gt; BinaryHeader,AddressInterval,AddressOffset,PackedVersion,BlockSize</li>
- *     <li>BinaryHeader --&gt; FieldNumber,EntryType,BinaryType,MissingOffset,MinLength,MaxLength,DataOffset</li>
- *     <li>SortedEntry --&gt; FieldNumber,EntryType,BinaryEntry,NumericEntry</li>
- *     <li>SortedSetEntry --&gt; EntryType,BinaryEntry,NumericEntry,NumericEntry</li>
- *     <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --&gt; {@link DataOutput#writeVInt VInt}</li>
- *     <li>EntryType,CompressionType --&gt; {@link DataOutput#writeByte Byte}</li>
- *     <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- *     <li>MinValue,GCD,MissingOffset,AddressOffset,DataOffset --&gt; {@link DataOutput#writeLong Int64}</li>
- *     <li>TableSize --&gt; {@link DataOutput#writeVInt vInt}</li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
- *   <p>Sorted fields have two entries: a BinaryEntry with the value metadata,
- *      and an ordinary NumericEntry for the document-to-ord metadata.</p>
- *   <p>SortedSet fields have three entries: a BinaryEntry with the value metadata,
- *      and two NumericEntries for the document-to-ord-index and ordinal list metadata.</p>
- *   <p>FieldNumber of -1 indicates the end of metadata.</p>
- *   <p>EntryType is a 0 (NumericEntry) or 1 (BinaryEntry)</p>
- *   <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
- *   <p>NumericType indicates how Numeric values will be compressed:
- *      <ul>
- *         <li>0 --&gt; delta-compressed. For each block of 16k integers, every integer is delta-encoded
- *             from the minimum value within the block. 
- *         <li>1 --&gt, gcd-compressed. When all integers share a common divisor, only quotients are stored
- *             using blocks of delta-encoded ints.
- *         <li>2 --&gt; table-compressed. When the number of unique numeric values is small and it would save space,
- *             a lookup table of unique values is written, followed by the ordinal for each document.
- *      </ul>
- *   <p>BinaryType indicates how Binary values will be stored:
- *      <ul>
- *         <li>0 --&gt; fixed-width. All values have the same length, addressing by multiplication. 
- *         <li>1 --&gt, variable-width. An address for each value is stored.
- *         <li>2 --&gt; prefix-compressed. An address to the start of every interval'th value is stored.
- *      </ul>
- *   <p>MinLength and MaxLength represent the min and max byte[] value lengths for Binary values.
- *      If they are equal, then all values are of a fixed size, and can be addressed as DataOffset + (docID * length).
- *      Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
- *      is written for the addresses.
- *   <p>MissingOffset points to a byte[] containing a bitset of all documents that had a value for the field.
- *      If its -1, then there are no missing values.
- *   <p>Checksum contains the CRC32 checksum of all bytes in the .dvm file up
- *      until the checksum. This is used to verify integrity of the file on opening the
- *      index.
- *   <li><a name="dvd" id="dvd"></a>
- *   <p>The DocValues data or .dvd file.</p>
- *   <p>For DocValues field, this stores the actual per-document data (the heavy-lifting)</p>
- *   <p>DocValues data (.dvd) --&gt; Header,&lt;NumericData | BinaryData | SortedData&gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>NumericData --&gt; DeltaCompressedNumerics | TableCompressedNumerics | GCDCompressedNumerics</li>
- *     <li>BinaryData --&gt;  {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
- *     <li>SortedData --&gt; {@link FST FST&lt;Int64&gt;}</li>
- *     <li>DeltaCompressedNumerics --&gt; {@link BlockPackedWriter BlockPackedInts(blockSize=16k)}</li>
- *     <li>TableCompressedNumerics --&gt; {@link PackedInts PackedInts}</li>
- *     <li>GCDCompressedNumerics --&gt; {@link BlockPackedWriter BlockPackedInts(blockSize=16k)}</li>
- *     <li>Addresses --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
- *   <p>SortedSet entries store the list of ordinals in their BinaryData as a
- *      sequences of increasing {@link DataOutput#writeVLong vLong}s, delta-encoded.</p>
- * </ol>
  * @deprecated Only for reading old 4.3-4.5 segments
- * @lucene.experimental
  */
 @Deprecated
 public class Lucene45DocValuesFormat extends DocValuesFormat {
@@ -177,7 +44,7 @@ public class Lucene45DocValuesFormat ext
   }
 
   @Override
-  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public final DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
     return new Lucene45DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
   }
   

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java Sun Oct  5 14:43:47 2014
@@ -63,13 +63,16 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LongValues;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.Version;
 import org.apache.lucene.util.packed.BlockPackedReader;
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
 import org.apache.lucene.util.packed.PackedInts;
 
-/** reader for {@link Lucene45DocValuesFormat} */
+/** 
+ * reader for 4.5 docvalues format
+ * @deprecated only for reading old 4.x segments
+ */
+@Deprecated
 class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
   private final Map<Integer,NumericEntry> numerics;
   private final Map<Integer,BinaryEntry> binaries;
@@ -94,6 +97,27 @@ class Lucene45DocValuesProducer extends 
   private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
   private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
   
+  private final boolean merging;
+  
+  // clone for merge: when merging we don't do any instances.put()s
+  Lucene45DocValuesProducer(Lucene45DocValuesProducer original) throws IOException {
+    assert Thread.holdsLock(original);
+    numerics = original.numerics;
+    binaries = original.binaries;
+    sortedSets = original.sortedSets;
+    ords = original.ords;
+    ordIndexes = original.ordIndexes;
+    ramBytesUsed = new AtomicLong(original.ramBytesUsed.get());
+    data = original.data.clone();
+    maxDoc = original.maxDoc;
+    version = original.version;
+    numFields = original.numFields;
+    lenientFieldInfoCheck = original.lenientFieldInfoCheck;
+    addressInstances.putAll(original.addressInstances);
+    ordIndexInstances.putAll(original.ordIndexInstances);
+    merging = true;
+  }
+  
   /** expert: instantiates a new reader */
   @SuppressWarnings("deprecation")
   protected Lucene45DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
@@ -103,6 +127,7 @@ class Lucene45DocValuesProducer extends 
     // read in the entries from the metadata file.
     ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
     this.maxDoc = state.segmentInfo.getDocCount();
+    merging = false;
     boolean success = false;
     try {
       version = CodecUtil.checkHeader(in, metaCodec, 
@@ -439,8 +464,10 @@ class Lucene45DocValuesProducer extends 
     if (addrInstance == null) {
       data.seek(bytes.addressesOffset);
       addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
-      addressInstances.put(field.number, addrInstance);
-      ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      if (!merging) {
+        addressInstances.put(field.number, addrInstance);
+        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      }
     }
     addresses = addrInstance;
     return addresses;
@@ -486,8 +513,10 @@ class Lucene45DocValuesProducer extends 
         size = 1L + bytes.count / interval;
       }
       addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-      addressInstances.put(field.number, addrInstance);
-      ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      if (!merging) {
+        addressInstances.put(field.number, addrInstance);
+        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      }
     }
     addresses = addrInstance;
     return addresses;
@@ -556,8 +585,10 @@ class Lucene45DocValuesProducer extends 
     if (ordIndexInstance == null) {
       data.seek(entry.offset);
       ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
-      ordIndexInstances.put(field.number, ordIndexInstance);
-      ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      if (!merging) {
+        ordIndexInstances.put(field.number, ordIndexInstance);
+        ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      }
     }
     ordIndex = ordIndexInstance;
     return ordIndex;
@@ -692,6 +723,11 @@ class Lucene45DocValuesProducer extends 
   }
 
   @Override
+  public synchronized DocValuesProducer getMergeInstance() throws IOException {
+    return new Lucene45DocValuesProducer(this);
+  }
+
+  @Override
   public void close() throws IOException {
     data.close();
   }

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java Sun Oct  5 14:43:47 2014
@@ -20,9 +20,9 @@ package org.apache.lucene.codecs.lucene4
 import java.io.IOException;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.codecs.NormsConsumer;
 import org.apache.lucene.codecs.NormsFormat;
@@ -30,6 +30,7 @@ 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.lucene40.Lucene40CompoundFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
@@ -39,19 +40,9 @@ import org.apache.lucene.codecs.perfield
 import org.apache.lucene.index.SegmentWriteState;
 
 /**
- * Implements the Lucene 4.6 index format, with configurable per-field postings
- * and docvalues formats.
- * <p>
- * If you want to reuse functionality of this codec in another codec, extend
- * {@link FilterCodec}.
- *
- * @see org.apache.lucene.codecs.lucene46 package documentation for file format details.
- * @lucene.experimental
+ * Implements the Lucene 4.6 index format
  * @deprecated Only for reading old 4.6-4.8 segments
  */
-// NOTE: if we make largish changes in a minor release, easier to just make Lucene46Codec or whatever
-// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
-// (it writes a minor version, etc).
 @Deprecated
 public class Lucene46Codec extends Codec {
   private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
@@ -59,6 +50,7 @@ public class Lucene46Codec extends Codec
   private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
   private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();
   private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
+  private final CompoundFormat compoundFormat = new Lucene40CompoundFormat();
   
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
     @Override
@@ -80,12 +72,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;
   }
 
@@ -108,6 +100,11 @@ public class Lucene46Codec extends Codec
   public final LiveDocsFormat liveDocsFormat() {
     return liveDocsFormat;
   }
+  
+  @Override
+  public CompoundFormat compoundFormat() {
+    return compoundFormat;
+  }
 
   /** Returns the postings format that should be used for writing 
    *  new segments of <code>field</code>.

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosFormat.java Sun Oct  5 14:43:47 2014
@@ -19,77 +19,15 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FieldInfosReader;
 import org.apache.lucene.codecs.FieldInfosWriter;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.store.DataOutput;
 
 /**
  * Lucene 4.6 Field Infos format.
- * <p>
- * <p>Field names are stored in the field info file, with suffix <tt>.fnm</tt>.</p>
- * <p>FieldInfos (.fnm) --&gt; Header,FieldsCount, &lt;FieldName,FieldNumber,
- * FieldBits,DocValuesBits,DocValuesGen,Attributes&gt; <sup>FieldsCount</sup>,Footer</p>
- * <p>Data types:
- * <ul>
- *   <li>Header --&gt; {@link CodecUtil#checkHeader CodecHeader}</li>
- *   <li>FieldsCount --&gt; {@link DataOutput#writeVInt VInt}</li>
- *   <li>FieldName --&gt; {@link DataOutput#writeString String}</li>
- *   <li>FieldBits, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
- *   <li>FieldNumber --&gt; {@link DataOutput#writeInt VInt}</li>
- *   <li>Attributes --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
- *   <li>DocValuesGen --&gt; {@link DataOutput#writeLong(long) Int64}</li>
- *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * </p>
- * Field Descriptions:
- * <ul>
- *   <li>FieldsCount: the number of fields in this file.</li>
- *   <li>FieldName: name of the field as a UTF-8 String.</li>
- *   <li>FieldNumber: the field's number. Note that unlike previous versions of
- *       Lucene, the fields are not numbered implicitly by their order in the
- *       file, instead explicitly.</li>
- *   <li>FieldBits: a byte containing field options.
- *       <ul>
- *         <li>The low-order bit is one for indexed fields, and zero for non-indexed
- *             fields.</li>
- *         <li>The second lowest-order bit is one for fields that have term vectors
- *             stored, and zero for fields without term vectors.</li>
- *         <li>If the third lowest order-bit is set (0x4), offsets are stored into
- *             the postings list in addition to positions.</li>
- *         <li>Fourth bit is unused.</li>
- *         <li>If the fifth lowest-order bit is set (0x10), norms are omitted for the
- *             indexed field.</li>
- *         <li>If the sixth lowest-order bit is set (0x20), payloads are stored for the
- *             indexed field.</li>
- *         <li>If the seventh lowest-order bit is set (0x40), term frequencies and
- *             positions omitted for the indexed field.</li>
- *         <li>If the eighth lowest-order bit is set (0x80), positions are omitted for the
- *             indexed field.</li>
- *       </ul>
- *    </li>
- *    <li>DocValuesBits: a byte containing per-document value types. The type
- *        recorded as two four-bit integers, with the high-order bits representing
- *        <code>norms</code> options, and the low-order bits representing 
- *        {@code DocValues} options. Each four-bit integer can be decoded as such:
- *        <ul>
- *          <li>0: no DocValues for this field.</li>
- *          <li>1: NumericDocValues. ({@link DocValuesType#NUMERIC})</li>
- *          <li>2: BinaryDocValues. ({@code DocValuesType#BINARY})</li>
- *          <li>3: SortedDocValues. ({@code DocValuesType#SORTED})</li>
- *        </ul>
- *    </li>
- *    <li>DocValuesGen is the generation count of the field's DocValues. If this is -1,
- *        there are no DocValues updates to that field. Anything above zero means there 
- *        are updates stored by {@link DocValuesFormat}.</li>
- *    <li>Attributes: a key-value map of codec-private attributes.</li>
- * </ul>
- *
- * @lucene.experimental
+ * @deprecated only for old 4.x segments
  */
+@Deprecated
 public final class Lucene46FieldInfosFormat extends FieldInfosFormat {
   private final FieldInfosReader reader = new Lucene46FieldInfosReader();
   private final FieldInfosWriter writer = new Lucene46FieldInfosWriter();
@@ -99,7 +37,7 @@ public final class Lucene46FieldInfosFor
   }
 
   @Override
-  public FieldInfosReader getFieldInfosReader() throws IOException {
+  public final FieldInfosReader getFieldInfosReader() throws IOException {
     return reader;
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosReader.java Sun Oct  5 14:43:47 2014
@@ -38,9 +38,9 @@ import org.apache.lucene.store.IndexInpu
 /**
  * Lucene 4.6 FieldInfos reader.
  * 
- * @lucene.experimental
- * @see Lucene46FieldInfosFormat
+ * @deprecated only for old 4.x segments
  */
+@Deprecated
 final class Lucene46FieldInfosReader extends FieldInfosReader {
 
   /** Sole constructor. */

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46FieldInfosWriter.java Sun Oct  5 14:43:47 2014
@@ -34,9 +34,9 @@ import org.apache.lucene.store.IOContext
 /**
  * Lucene 4.6 FieldInfos writer.
  * 
- * @see Lucene46FieldInfosFormat
- * @lucene.experimental
+ * @deprecated only for old 4.x segments
  */
+@Deprecated
 final class Lucene46FieldInfosWriter extends FieldInfosWriter {
   
   /** Sole constructor. */

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoFormat.java Sun Oct  5 14:43:47 2014
@@ -17,54 +17,16 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
-import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.SegmentInfoReader;
 import org.apache.lucene.codecs.SegmentInfoWriter;
-import org.apache.lucene.index.IndexWriter; // javadocs
-import org.apache.lucene.index.SegmentInfo; // javadocs
-import org.apache.lucene.index.SegmentInfos; // javadocs
-import org.apache.lucene.store.DataOutput; // javadocs
+import org.apache.lucene.index.SegmentInfo;
 
 /**
  * Lucene 4.6 Segment info format.
- * <p>
- * Files:
- * <ul>
- *   <li><tt>.si</tt>: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Files, Footer
- * </ul>
- * </p>
- * Data types:
- * <p>
- * <ul>
- *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- *   <li>SegSize --&gt; {@link DataOutput#writeInt Int32}</li>
- *   <li>SegVersion --&gt; {@link DataOutput#writeString String}</li>
- *   <li>Files --&gt; {@link DataOutput#writeStringSet Set&lt;String&gt;}</li>
- *   <li>Diagnostics --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
- *   <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
- *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- * </ul>
- * </p>
- * Field Descriptions:
- * <p>
- * <ul>
- *   <li>SegVersion is the code version that created the segment.</li>
- *   <li>SegSize is the number of documents contained in the segment index.</li>
- *   <li>IsCompoundFile records whether the segment is written as a compound file or
- *       not. If this is -1, the segment is not a compound file. If it is 1, the segment
- *       is a compound file.</li>
- *   <li>The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid,
- *       for each segment it creates. It includes metadata like the current Lucene
- *       version, OS, Java version, why the segment was created (merge, flush,
- *       addIndexes), etc.</li>
- *   <li>Files is a list of files referred to by this segment.</li>
- * </ul>
- * </p>
- * 
- * @see SegmentInfos
- * @lucene.experimental
+ * @deprecated only for old 4.x segments
  */
+@Deprecated
 public class Lucene46SegmentInfoFormat extends SegmentInfoFormat {
   private final SegmentInfoReader reader = new Lucene46SegmentInfoReader();
 
@@ -73,7 +35,7 @@ public class Lucene46SegmentInfoFormat e
   }
   
   @Override
-  public SegmentInfoReader getSegmentInfoReader() {
+  public final SegmentInfoReader getSegmentInfoReader() {
     return reader;
   }
 
@@ -83,7 +45,7 @@ public class Lucene46SegmentInfoFormat e
   }
 
   /** File extension used to store {@link SegmentInfo}. */
-  public final static String SI_EXTENSION = "si";
+  final static String SI_EXTENSION = "si";
   static final String CODEC_NAME = "Lucene46SegmentInfo";
   static final int VERSION_START = 0;
   static final int VERSION_CHECKSUM = 1;

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoReader.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene46/Lucene46SegmentInfoReader.java Sun Oct  5 14:43:47 2014
@@ -33,12 +33,11 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.util.Version;
 
 /**
- * Lucene 4.6 implementation of {@link SegmentInfoReader}.
- * 
- * @see Lucene46SegmentInfoFormat
- * @lucene.experimental
+ * Lucene 4.6 segment infos reader
+ * @deprecated only for old 4.x segments
  */
-public class Lucene46SegmentInfoReader extends SegmentInfoReader {
+@Deprecated
+final class Lucene46SegmentInfoReader extends SegmentInfoReader {
 
   /** Sole constructor. */
   public Lucene46SegmentInfoReader() {

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49Codec.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49Codec.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49Codec.java Sun Oct  5 14:43:47 2014
@@ -20,9 +20,9 @@ package org.apache.lucene.codecs.lucene4
 import java.io.IOException;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundFormat;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
-import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.codecs.NormsConsumer;
 import org.apache.lucene.codecs.NormsFormat;
@@ -30,6 +30,7 @@ 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.lucene40.Lucene40CompoundFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
 import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
@@ -40,24 +41,17 @@ import org.apache.lucene.codecs.perfield
 import org.apache.lucene.index.SegmentWriteState;
 
 /**
- * Implements the Lucene 4.9 index format, with configurable per-field postings
- * and docvalues formats.
- * <p>
- * If you want to reuse functionality of this codec in another codec, extend
- * {@link FilterCodec}.
- *
- * @see org.apache.lucene.codecs.lucene49 package documentation for file format details.
- * @lucene.experimental
+ * Implements the Lucene 4.9 index format
+ * @deprecated only for old 4.x segments
  */
-// NOTE: if we make largish changes in a minor release, easier to just make Lucene410Codec or whatever
-// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
-// (it writes a minor version, etc).
+@Deprecated
 public class Lucene49Codec extends Codec {
   private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
   private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
   private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
   private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();
   private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
+  private final CompoundFormat compoundFormat = new Lucene40CompoundFormat();
   
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
     @Override
@@ -79,12 +73,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;
   }
 
@@ -107,6 +101,11 @@ public class Lucene49Codec extends Codec
   public final LiveDocsFormat liveDocsFormat() {
     return liveDocsFormat;
   }
+  
+  @Override
+  public CompoundFormat compoundFormat() {
+    return compoundFormat;
+  }
 
   /** Returns the postings format that should be used for writing 
    *  new segments of <code>field</code>.

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesConsumer.java Sun Oct  5 14:43:47 2014
@@ -40,7 +40,11 @@ import org.apache.lucene.util.packed.Dir
 import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
 import org.apache.lucene.util.packed.PackedInts;
 
-/** writer for {@link Lucene49DocValuesFormat} */
+/** 
+ * writer for 4.9 docvalues format
+ * @deprecated only for old 4.x segments
+ */
+@Deprecated
 class Lucene49DocValuesConsumer extends DocValuesConsumer implements Closeable {
 
   static final int BLOCK_SIZE = 16384;

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesFormat.java Sun Oct  5 14:43:47 2014
@@ -19,151 +19,17 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.util.SmallFloat;
-import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.packed.DirectWriter;
-import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
 
 /**
  * Lucene 4.9 DocValues format.
- * <p>
- * Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
- * <p>
- * {@link DocValuesType#NUMERIC NUMERIC}:
- * <ul>
- *    <li>Delta-compressed: per-document integers written as deltas from the minimum value,
- *        compressed with bitpacking. For more information, see {@link DirectWriter}.
- *    <li>Table-compressed: when the number of unique values is very small (&lt; 256), and
- *        when there are unused "gaps" in the range of values used (such as {@link SmallFloat}), 
- *        a lookup table is written instead. Each per-document entry is instead the ordinal 
- *        to this table, and those ordinals are compressed with bitpacking ({@link DirectWriter}). 
- *    <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest
- *        common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
- *    <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are written
- *        as blocks of bitpacked integers, encoding the deviation from the expected delta.
- * </ul>
- * <p>
- * {@link DocValuesType#BINARY BINARY}:
- * <ul>
- *    <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
- *        Each document's value can be addressed directly with multiplication ({@code docID * length}). 
- *    <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses 
- *        for each document. The addresses are written as Monotonic-compressed numerics.
- *    <li>Prefix-compressed Binary: values are written in chunks of 16, with the first value written
- *        completely and other values sharing prefixes. chunk addresses are written as Monotonic-compressed
- *        numerics.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED SORTED}:
- * <ul>
- *    <li>Sorted: a mapping of ordinals to deduplicated terms is written as Prefix-Compressed Binary, 
- *        along with the per-document ordinals written using one of the numeric strategies above.
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED_SET SORTED_SET}:
- * <ul>
- *    <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Prefix-Compressed Binary, 
- *        an ordinal list and per-document index into this list are written using the numeric strategies 
- *        above. 
- * </ul>
- * <p>
- * {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
- * <ul>
- *    <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
- *        strategies above.
- * </ul>
- * <p>
- * Files:
- * <ol>
- *   <li><tt>.dvd</tt>: DocValues data</li>
- *   <li><tt>.dvm</tt>: DocValues metadata</li>
- * </ol>
- * <ol>
- *   <li><a name="dvm" id="dvm"></a>
- *   <p>The DocValues metadata or .dvm file.</p>
- *   <p>For DocValues field, this stores metadata, such as the offset into the 
- *      DocValues data (.dvd)</p>
- *   <p>DocValues metadata (.dvm) --&gt; Header,&lt;Entry&gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>Entry --&gt; NumericEntry | BinaryEntry | SortedEntry | SortedSetEntry | SortedNumericEntry</li>
- *     <li>NumericEntry --&gt; GCDNumericEntry | TableNumericEntry | DeltaNumericEntry</li>
- *     <li>GCDNumericEntry --&gt; NumericHeader,MinValue,GCD,BitsPerValue</li>
- *     <li>TableNumericEntry --&gt; NumericHeader,TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,BitsPerValue</li>
- *     <li>DeltaNumericEntry --&gt; NumericHeader,MinValue,BitsPerValue</li>
- *     <li>MonotonicNumericEntry --&gt; NumericHeader,PackedVersion,BlockSize</li>
- *     <li>NumericHeader --&gt; FieldNumber,EntryType,NumericType,MissingOffset,DataOffset,Count,EndOffset</li>
- *     <li>BinaryEntry --&gt; FixedBinaryEntry | VariableBinaryEntry | PrefixBinaryEntry</li>
- *     <li>FixedBinaryEntry --&gt; BinaryHeader</li>
- *     <li>VariableBinaryEntry --&gt; BinaryHeader,AddressOffset,PackedVersion,BlockSize</li>
- *     <li>PrefixBinaryEntry --&gt; BinaryHeader,AddressInterval,AddressOffset,PackedVersion,BlockSize</li>
- *     <li>BinaryHeader --&gt; FieldNumber,EntryType,BinaryType,MissingOffset,MinLength,MaxLength,DataOffset</li>
- *     <li>SortedEntry --&gt; FieldNumber,EntryType,BinaryEntry,NumericEntry</li>
- *     <li>SortedSetEntry --&gt; EntryType,BinaryEntry,NumericEntry,NumericEntry</li>
- *     <li>SortedNumericEntry --&gt; EntryType,NumericEntry,NumericEntry</li>
- *     <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --&gt; {@link DataOutput#writeVInt VInt}</li>
- *     <li>EntryType,CompressionType --&gt; {@link DataOutput#writeByte Byte}</li>
- *     <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
- *     <li>MinValue,GCD,MissingOffset,AddressOffset,DataOffset,EndOffset --&gt; {@link DataOutput#writeLong Int64}</li>
- *     <li>TableSize,BitsPerValue --&gt; {@link DataOutput#writeVInt vInt}</li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
- *   <p>Sorted fields have two entries: a BinaryEntry with the value metadata,
- *      and an ordinary NumericEntry for the document-to-ord metadata.</p>
- *   <p>SortedSet fields have three entries: a BinaryEntry with the value metadata,
- *      and two NumericEntries for the document-to-ord-index and ordinal list metadata.</p>
- *   <p>SortedNumeric fields have two entries: A NumericEntry with the value metadata,
- *      and a numeric entry with the document-to-value index.</p>
- *   <p>FieldNumber of -1 indicates the end of metadata.</p>
- *   <p>EntryType is a 0 (NumericEntry) or 1 (BinaryEntry)</p>
- *   <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
- *   <p>EndOffset is the pointer to the end of the data in the DocValues data (.dvd)</p>
- *   <p>NumericType indicates how Numeric values will be compressed:
- *      <ul>
- *         <li>0 --&gt; delta-compressed. For each block of 16k integers, every integer is delta-encoded
- *             from the minimum value within the block. 
- *         <li>1 --&gt, gcd-compressed. When all integers share a common divisor, only quotients are stored
- *             using blocks of delta-encoded ints.
- *         <li>2 --&gt; table-compressed. When the number of unique numeric values is small and it would save space,
- *             a lookup table of unique values is written, followed by the ordinal for each document.
- *      </ul>
- *   <p>BinaryType indicates how Binary values will be stored:
- *      <ul>
- *         <li>0 --&gt; fixed-width. All values have the same length, addressing by multiplication. 
- *         <li>1 --&gt, variable-width. An address for each value is stored.
- *         <li>2 --&gt; prefix-compressed. An address to the start of every interval'th value is stored.
- *      </ul>
- *   <p>MinLength and MaxLength represent the min and max byte[] value lengths for Binary values.
- *      If they are equal, then all values are of a fixed size, and can be addressed as DataOffset + (docID * length).
- *      Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
- *      is written for the addresses.
- *   <p>MissingOffset points to a byte[] containing a bitset of all documents that had a value for the field.
- *      If its -1, then there are no missing values.
- *   <p>Checksum contains the CRC32 checksum of all bytes in the .dvm file up
- *      until the checksum. This is used to verify integrity of the file on opening the
- *      index.
- *   <li><a name="dvd" id="dvd"></a>
- *   <p>The DocValues data or .dvd file.</p>
- *   <p>For DocValues field, this stores the actual per-document data (the heavy-lifting)</p>
- *   <p>DocValues data (.dvd) --&gt; Header,&lt;NumericData | BinaryData | SortedData&gt;<sup>NumFields</sup>,Footer</p>
- *   <ul>
- *     <li>NumericData --&gt; DeltaCompressedNumerics | TableCompressedNumerics | GCDCompressedNumerics</li>
- *     <li>BinaryData --&gt;  {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
- *     <li>SortedData --&gt; {@link FST FST&lt;Int64&gt;}</li>
- *     <li>DeltaCompressedNumerics,TableCompressedNumerics,GCDCompressedNumerics --&gt; {@link DirectWriter PackedInts}</li>
- *     <li>Addresses --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</li>
- *     <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
- *   </ul>
- * </ol>
- * @lucene.experimental
+ * @deprecated only for old 4.x segments
  */
+@Deprecated
 public class Lucene49DocValuesFormat extends DocValuesFormat {
 
   /** Sole Constructor */
@@ -177,7 +43,7 @@ public class Lucene49DocValuesFormat ext
   }
 
   @Override
-  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+  public final DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
     return new Lucene49DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
   }
   

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene49/Lucene49DocValuesProducer.java Sun Oct  5 14:43:47 2014
@@ -67,56 +67,75 @@ import org.apache.lucene.util.RamUsageEs
 import org.apache.lucene.util.packed.DirectReader;
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
 
-/** reader for {@link Lucene49DocValuesFormat} */
+/** 
+ * reader for 4.9 docvalues format
+ * @deprecated only for 4.x segments 
+ */
+@Deprecated
 class Lucene49DocValuesProducer extends DocValuesProducer implements Closeable {
-  private final Map<String,NumericEntry> numerics;
-  private final Map<String,BinaryEntry> binaries;
-  private final Map<String,SortedSetEntry> sortedSets;
-  private final Map<String,SortedSetEntry> sortedNumerics;
-  private final Map<String,NumericEntry> ords;
-  private final Map<String,NumericEntry> ordIndexes;
+  private final Map<String,NumericEntry> numerics = new HashMap<>();
+  private final Map<String,BinaryEntry> binaries = new HashMap<>();
+  private final Map<String,SortedSetEntry> sortedSets = new HashMap<>();
+  private final Map<String,SortedSetEntry> sortedNumerics = new HashMap<>();
+  private final Map<String,NumericEntry> ords = new HashMap<>();
+  private final Map<String,NumericEntry> ordIndexes = new HashMap<>();
   private final AtomicLong ramBytesUsed;
   private final IndexInput data;
   private final int numFields;
   private final int maxDoc;
-  private final int version;
 
   // memory-resident structures
   private final Map<String,MonotonicBlockPackedReader> addressInstances = new HashMap<>();
   private final Map<String,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<>();
   
+  private final boolean merging;
+  
+  // clone for merge: when merging we don't do any instances.put()s
+  Lucene49DocValuesProducer(Lucene49DocValuesProducer original) throws IOException {
+    assert Thread.holdsLock(original);
+    numerics.putAll(original.numerics);
+    binaries.putAll(original.binaries);
+    sortedSets.putAll(original.sortedSets);
+    sortedNumerics.putAll(original.sortedNumerics);
+    ords.putAll(original.ords);
+    ordIndexes.putAll(original.ordIndexes);
+    ramBytesUsed = new AtomicLong(original.ramBytesUsed());
+    data = original.data.clone();
+    numFields = original.numFields;
+    maxDoc = original.maxDoc;
+    addressInstances.putAll(original.addressInstances);
+    ordIndexInstances.putAll(original.ordIndexInstances);
+    merging = true;
+  }
+  
   /** expert: instantiates a new reader */
   Lucene49DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
-    // read in the entries from the metadata file.
-    ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context);
     this.maxDoc = state.segmentInfo.getDocCount();
-    boolean success = false;
-    try {
-      version = CodecUtil.checkHeader(in, metaCodec, 
-                                      Lucene49DocValuesFormat.VERSION_START,
-                                      Lucene49DocValuesFormat.VERSION_CURRENT);
-      numerics = new HashMap<>();
-      ords = new HashMap<>();
-      ordIndexes = new HashMap<>();
-      binaries = new HashMap<>();
-      sortedSets = new HashMap<>();
-      sortedNumerics = new HashMap<>();
-      numFields = readFields(in, state.fieldInfos);
-
-      CodecUtil.checkFooter(in);
-      success = true;
-    } finally {
-      if (success) {
-        IOUtils.close(in);
-      } else {
-        IOUtils.closeWhileHandlingException(in);
+    merging = false;
+    
+    int version = -1;
+    int numFields = -1;
+    
+    // read in the entries from the metadata file.
+    try (ChecksumIndexInput in = state.directory.openChecksumInput(metaName, state.context)) {
+      Throwable priorE = null;
+      try {
+        version = CodecUtil.checkHeader(in, metaCodec, 
+                                            Lucene49DocValuesFormat.VERSION_START,
+                                            Lucene49DocValuesFormat.VERSION_CURRENT);
+        numFields = readFields(in, state.fieldInfos);
+      } catch (Throwable exception) {
+        priorE = exception;
+      } finally {
+        CodecUtil.checkFooter(in, priorE);
       }
     }
+    this.numFields = numFields;
 
     String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
     this.data = state.directory.openInput(dataName, state.context);
-    success = false;
+    boolean success = false;
     try {
       final int version2 = CodecUtil.checkHeader(data, dataCodec, 
                                                  Lucene49DocValuesFormat.VERSION_START,
@@ -443,8 +462,10 @@ class Lucene49DocValuesProducer extends 
     if (addrInstance == null) {
       data.seek(bytes.addressesOffset);
       addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count+1, false);
-      addressInstances.put(field.name, addrInstance);
-      ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      if (!merging) {
+        addressInstances.put(field.name, addrInstance);
+        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      }
     }
     addresses = addrInstance;
     return addresses;
@@ -489,8 +510,10 @@ class Lucene49DocValuesProducer extends 
         size = 1L + bytes.count / interval;
       }
       addrInstance = MonotonicBlockPackedReader.of(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-      addressInstances.put(field.name, addrInstance);
-      ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      if (!merging) {
+        addressInstances.put(field.name, addrInstance);
+        ramBytesUsed.addAndGet(addrInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      }
     }
     addresses = addrInstance;
     return addresses;
@@ -556,8 +579,10 @@ class Lucene49DocValuesProducer extends 
     if (ordIndexInstance == null) {
       data.seek(entry.offset);
       ordIndexInstance = MonotonicBlockPackedReader.of(data, entry.packedIntsVersion, entry.blockSize, entry.count+1, false);
-      ordIndexInstances.put(field.name, ordIndexInstance);
-      ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      if (!merging) {
+        ordIndexInstances.put(field.name, ordIndexInstance);
+        ramBytesUsed.addAndGet(ordIndexInstance.ramBytesUsed() + RamUsageEstimator.NUM_BYTES_INT);
+      }
     }
     ordIndex = ordIndexInstance;
     return ordIndex;
@@ -729,6 +754,11 @@ class Lucene49DocValuesProducer extends 
     data.close();
   }
   
+  @Override
+  public synchronized DocValuesProducer getMergeInstance() throws IOException {
+    return new Lucene49DocValuesProducer(this);
+  }
+
   /** metadata entry for a numeric docvalues field */
   static class NumericEntry {
     private NumericEntry() {}

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java Sun Oct  5 14:43:47 2014
@@ -26,18 +26,21 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.MissingOrdRemapper;
 import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosReader.LegacyDocValuesType;
-import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.PackedInts;
 
-class Lucene40DocValuesWriter extends DocValuesConsumer {
+/**
+ * Writer for 4.0 docvalues format
+ * @deprecated for test purposes only
+ */
+@Deprecated
+final class Lucene40DocValuesWriter extends DocValuesConsumer {
   private final Directory dir;
   private final SegmentWriteState state;
   private final String legacyKey;
@@ -47,7 +50,7 @@ class Lucene40DocValuesWriter extends Do
   Lucene40DocValuesWriter(SegmentWriteState state, String filename, String legacyKey) throws IOException {
     this.state = state;
     this.legacyKey = legacyKey;
-    this.dir = new CompoundFileDirectory(state.directory, filename, state.context, true);
+    this.dir = new Lucene40CompoundReader(state.directory, filename, state.context, true);
   }
   
   @Override

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosWriter.java Sun Oct  5 14:43:47 2014
@@ -33,13 +33,11 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.IOUtils;
 
 /**
- * Lucene 4.0 FieldInfos writer.
- * 
- * @see Lucene40FieldInfosFormat
- * @lucene.experimental
+ * Writer for 4.0 fieldinfos format
+ * @deprecated for test purposes only
  */
 @Deprecated
-public class Lucene40FieldInfosWriter extends FieldInfosWriter {
+public final class Lucene40FieldInfosWriter extends FieldInfosWriter {
 
   /** Sole constructor. */
   public Lucene40FieldInfosWriter() {

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java Sun Oct  5 14:43:47 2014
@@ -37,12 +37,11 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 /**
- * Concrete class that writes the 4.0 frq/prx postings format.
- * 
- * @see Lucene40PostingsFormat
- * @lucene.experimental 
+ * Writer for 4.0 postings format
+ * @deprecated for test purposes only
  */
-public final class Lucene40PostingsWriter extends PushPostingsWriterBase {
+@Deprecated
+final class Lucene40PostingsWriter extends PushPostingsWriterBase {
 
   final IndexOutput freqOut;
   final IndexOutput proxOut;

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWCodec.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWCodec.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWCodec.java Sun Oct  5 14:43:47 2014
@@ -28,8 +28,11 @@ import org.apache.lucene.codecs.TermVect
  * limitations under the License.
  */
 
-/** Read-write version of Lucene40Codec for testing */
-@SuppressWarnings("deprecation")
+/**
+ * Read-write version of 4.0 codec for testing
+ * @deprecated for test purposes only
+ */
+@Deprecated
 public final class Lucene40RWCodec extends Lucene40Codec {
   
   private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat() {

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWDocValuesFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWDocValuesFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWDocValuesFormat.java Sun Oct  5 14:43:47 2014
@@ -23,15 +23,18 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
 
-/** Read-write version of {@link Lucene40DocValuesFormat} for testing */
-@SuppressWarnings("deprecation")
-public class Lucene40RWDocValuesFormat extends Lucene40DocValuesFormat {
+/**
+ * Read-write version of 4.0 docvalues format for testing
+ * @deprecated for test purposes only
+ */
+@Deprecated
+public final class Lucene40RWDocValuesFormat extends Lucene40DocValuesFormat {
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     String filename = IndexFileNames.segmentFileName(state.segmentInfo.name, 
           "dv", 
-          IndexFileNames.COMPOUND_FILE_EXTENSION);
+          Lucene40CompoundFormat.COMPOUND_FILE_EXTENSION);
     return new Lucene40DocValuesWriter(state, filename, Lucene40FieldInfosReader.LEGACY_DV_TYPE_KEY);
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWNormsFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWNormsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWNormsFormat.java Sun Oct  5 14:43:47 2014
@@ -24,15 +24,18 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
 
-/** Read-write version of {@link Lucene40NormsFormat} for testing */
-@SuppressWarnings("deprecation")
-public class Lucene40RWNormsFormat extends Lucene40NormsFormat {
+/**
+ * Read-write version of 4.0 norms format for testing
+ * @deprecated for test purposes only
+ */
+@Deprecated
+public final class Lucene40RWNormsFormat extends Lucene40NormsFormat {
 
   @Override
   public NormsConsumer normsConsumer(SegmentWriteState state) throws IOException {
     String filename = IndexFileNames.segmentFileName(state.segmentInfo.name, 
         "nrm", 
-        IndexFileNames.COMPOUND_FILE_EXTENSION);
+        Lucene40CompoundFormat.COMPOUND_FILE_EXTENSION);
     final Lucene40DocValuesWriter impl = new Lucene40DocValuesWriter(state, filename, Lucene40FieldInfosReader.LEGACY_NORM_TYPE_KEY);
     return new NormsConsumer() {
       @Override

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java Sun Oct  5 14:43:47 2014
@@ -23,14 +23,19 @@ import org.apache.lucene.codecs.FieldsCo
 import org.apache.lucene.codecs.PostingsWriterBase;
 import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.util.LuceneTestCase;
 
 /**
- * Read-write version of {@link Lucene40PostingsFormat} for testing.
+ * Read-write version of 4.0 postings format for testing
+ * @deprecated for test purposes only
  */
-@SuppressWarnings("deprecation")
-public class Lucene40RWPostingsFormat extends Lucene40PostingsFormat {
+@Deprecated
+public final class Lucene40RWPostingsFormat extends Lucene40PostingsFormat {
   
+  /** minimum items (terms or sub-blocks) per block for 4.0 BlockTree */
+  final static int MIN_BLOCK_SIZE = 25;
+  /** maximum items (terms or sub-blocks) per block for 4.0 BlockTree */
+  final static int MAX_BLOCK_SIZE = 48;
+
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     PostingsWriterBase docs = new Lucene40PostingsWriter(state);
@@ -41,7 +46,7 @@ public class Lucene40RWPostingsFormat ex
     // Or... you must make a new Codec for this?
     boolean success = false;
     try {
-      FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, minBlockSize, maxBlockSize);
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
       success = true;
       return ret;
     } finally {

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWSegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWSegmentInfoFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWSegmentInfoFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWSegmentInfoFormat.java Sun Oct  5 14:43:47 2014
@@ -19,8 +19,12 @@ package org.apache.lucene.codecs.lucene4
 
 import org.apache.lucene.codecs.SegmentInfoWriter;
 
-/** read-write version of 4.0 segmentinfos for testing */
-public class Lucene40RWSegmentInfoFormat extends Lucene40SegmentInfoFormat {
+/**
+ * Read-write version of 4.0 segmentinfo format for testing
+ * @deprecated for test purposes only
+ */
+@Deprecated
+public final class Lucene40RWSegmentInfoFormat extends Lucene40SegmentInfoFormat {
 
   @Override
   public SegmentInfoWriter getSegmentInfoWriter() {

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWStoredFieldsFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWStoredFieldsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWStoredFieldsFormat.java Sun Oct  5 14:43:47 2014
@@ -24,10 +24,12 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 
-/** 
- * Simulates writing Lucene 4.0 Stored Fields Format.
- */ 
-public class Lucene40RWStoredFieldsFormat extends Lucene40StoredFieldsFormat {
+/**
+ * Read-write version of 4.0 stored fields format for testing
+ * @deprecated for test purposes only
+ */
+@Deprecated
+final class Lucene40RWStoredFieldsFormat extends Lucene40StoredFieldsFormat {
 
   @Override
   public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWTermVectorsFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWTermVectorsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40RWTermVectorsFormat.java Sun Oct  5 14:43:47 2014
@@ -23,12 +23,13 @@ import org.apache.lucene.codecs.TermVect
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.LuceneTestCase;
 
-/** 
- * Simulates writing Lucene 4.0 Stored Fields Format.
- */ 
-public class Lucene40RWTermVectorsFormat extends Lucene40TermVectorsFormat {
+/**
+ * Read-write version of 4.0 term vectors format for testing
+ * @deprecated for test purposes only
+ */
+@Deprecated
+public final class Lucene40RWTermVectorsFormat extends Lucene40TermVectorsFormat {
 
   @Override
   public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoWriter.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoWriter.java Sun Oct  5 14:43:47 2014
@@ -31,13 +31,11 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.IOUtils;
 
 /**
- * Lucene 4.0 implementation of {@link SegmentInfoWriter}.
- * 
- * @see Lucene40SegmentInfoFormat
- * @lucene.experimental
+ * writer for 4.0 segmentinfos for testing
+ * @deprecated for test purposes only
  */
 @Deprecated
-public class Lucene40SegmentInfoWriter extends SegmentInfoWriter {
+public final class Lucene40SegmentInfoWriter extends SegmentInfoWriter {
 
   /** Sole constructor. */
   public Lucene40SegmentInfoWriter() {

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40SkipListWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40SkipListWriter.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40SkipListWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40SkipListWriter.java Sun Oct  5 14:43:47 2014
@@ -25,14 +25,11 @@ import org.apache.lucene.codecs.MultiLev
 
 
 /**
- * Implements the skip list writer for the 4.0 posting list format
- * that stores positions and payloads.
- * 
- * @see Lucene40PostingsFormat
- * @deprecated Only for reading old 4.0 segments
+ * Writer of 4.0 skip lists for testing
+ * @deprecated for test purposes only
  */
 @Deprecated
-public class Lucene40SkipListWriter extends MultiLevelSkipListWriter {
+final class Lucene40SkipListWriter extends MultiLevelSkipListWriter {
   private int[] lastSkipDoc;
   private int[] lastSkipPayloadLength;
   private int[] lastSkipOffsetLength;

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java Sun Oct  5 14:43:47 2014
@@ -34,15 +34,12 @@ import org.apache.lucene.util.IOUtils;
 import static org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsReader.*;
 
 
-/** 
- * Class responsible for writing stored document fields.
- * <p/>
- * It uses &lt;segment&gt;.fdt and &lt;segment&gt;.fdx; files.
- * 
- * @see Lucene40StoredFieldsFormat
- * @lucene.experimental 
+/**
+ * Writer for 4.0 stored fields format for testing
+ * @deprecated for test purposes only
  */
-public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
+@Deprecated
+final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
 
   private final Directory directory;
   private final String segment;

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java Sun Oct  5 14:43:47 2014
@@ -36,24 +36,12 @@ import org.apache.lucene.util.StringHelp
 
 import static org.apache.lucene.codecs.lucene40.Lucene40TermVectorsReader.*;
 
-
-// TODO: make a new 4.0 TV format that encodes better
-//   - use startOffset (not endOffset) as base for delta on
-//     next startOffset because today for syns or ngrams or
-//     WDF or shingles etc. we are encoding negative vints
-//     (= slow, 5 bytes per)
-//   - if doc has no term vectors, write 0 into the tvx
-//     file; saves a seek to tvd only to read a 0 vint (and
-//     saves a byte in tvd)
-
 /**
- * Lucene 4.0 Term Vectors writer.
- * <p>
- * It writes .tvd, .tvf, and .tvx files.
- * 
- * @see Lucene40TermVectorsFormat
+ * Writer for 4.0 term vectors format for testing
+ * @deprecated for test purposes only
  */
-public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
+@Deprecated
+final class Lucene40TermVectorsWriter extends TermVectorsWriter {
   private final Directory directory;
   private final String segment;
   private IndexOutput tvx = null, tvd = null, tvf = null;

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestBitVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestBitVector.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestBitVector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestBitVector.java Sun Oct  5 14:43:47 2014
@@ -24,7 +24,6 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.TestUtil;
 
 /**
  * <code>TestBitVector</code> tests the <code>BitVector</code>, obviously.

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestLucene40StoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestLucene40StoredFieldsFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestLucene40StoredFieldsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestLucene40StoredFieldsFormat.java Sun Oct  5 14:43:47 2014
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene4
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
-import org.junit.BeforeClass;
 
 public class TestLucene40StoredFieldsFormat extends BaseStoredFieldsFormatTestCase {
   

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestLucene40TermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestLucene40TermVectorsFormat.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestLucene40TermVectorsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene40/TestLucene40TermVectorsFormat.java Sun Oct  5 14:43:47 2014
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene4
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
-import org.junit.BeforeClass;
 
 public class TestLucene40TermVectorsFormat extends BaseTermVectorsFormatTestCase {
   

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41RWCodec.java Sun Oct  5 14:43:47 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
@@ -35,11 +34,12 @@ import org.apache.lucene.util.LuceneTest
  */
 
 /**
- * Read-write version of {@link Lucene41Codec} for testing.
+ * Read-write version of 4.1 codec for testing
+ * @deprecated for test purposes only
  */
-@SuppressWarnings("deprecation")
-public class Lucene41RWCodec extends Lucene41Codec {
-  private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
+@Deprecated
+public final class Lucene41RWCodec extends Lucene41Codec {
+  private final StoredFieldsFormat fieldsFormat = new Lucene41RWStoredFieldsFormat();
   private final FieldInfosFormat fieldInfos = new Lucene40FieldInfosFormat() {
     @Override
     public FieldInfosWriter getFieldInfosWriter() throws IOException {

Copied: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java (from r1629499, lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java?p2=lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java&p1=lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java&r1=1629499&r2=1629501&rev=1629501&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsWriter.java Sun Oct  5 14:43:47 2014
@@ -41,7 +41,7 @@ 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.index.IndexableField;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -229,7 +229,7 @@ final class Lucene41StoredFieldsWriter e
   }
 
   @Override
-  public void writeField(FieldInfo info, StorableField field)
+  public void writeField(FieldInfo info, IndexableField field)
       throws IOException {
 
     ++numStoredFieldsInDoc;

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java Sun Oct  5 14:43:47 2014
@@ -34,7 +34,6 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.MathUtil;
 import org.apache.lucene.util.fst.Builder;
@@ -58,9 +57,11 @@ import static org.apache.lucene.codecs.l
 import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.UNCOMPRESSED;
 
 /**
- * Writer for {@link Lucene42DocValuesFormat}
+ * Writer for 4.2 docvalues format for testing
+ * @deprecated for test purposes only
  */
-class Lucene42DocValuesConsumer extends DocValuesConsumer {
+@Deprecated
+final class Lucene42DocValuesConsumer extends DocValuesConsumer {
   final IndexOutput data, meta;
   final int maxDoc;
   final float acceptableOverheadRatio;

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java Sun Oct  5 14:43:47 2014
@@ -33,10 +33,8 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.IOUtils;
 
 /**
- * Lucene 4.2 FieldInfos writer.
- * 
- * @see Lucene42FieldInfosFormat
- * @lucene.experimental
+ * Writer for 4.2 fieldinfos format for testing
+ * @deprecated for test purposes only
  */
 @Deprecated
 public final class Lucene42FieldInfosWriter extends FieldInfosWriter {

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42NormsConsumer.java Sun Oct  5 14:43:47 2014
@@ -36,9 +36,11 @@ import org.apache.lucene.util.packed.Pac
 import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesProducer.VERSION_CURRENT;
 
 /**
- * Writer for {@link Lucene42NormsFormat}
+ * Writer for 4.2 norms format for testing
+ * @deprecated for test purposes only
  */
-class Lucene42NormsConsumer extends NormsConsumer { 
+@Deprecated
+final class Lucene42NormsConsumer extends NormsConsumer { 
   static final byte NUMBER = 0;
 
   static final int BLOCK_SIZE = 4096;

Modified: lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java?rev=1629501&r1=1629500&r2=1629501&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java (original)
+++ lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene42/Lucene42RWCodec.java Sun Oct  5 14:43:47 2014
@@ -24,17 +24,21 @@ 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.util.LuceneTestCase;
+import org.apache.lucene.codecs.lucene41.Lucene41RWStoredFieldsFormat;
 
 /**
- * Read-write version of {@link Lucene42Codec} for testing.
+ * Read-Write version of 4.2 codec for testing
+ * @deprecated for test purposes only
  */
-@SuppressWarnings("deprecation")
-public class Lucene42RWCodec extends Lucene42Codec {
+@Deprecated
+public final class Lucene42RWCodec extends Lucene42Codec {
 
   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;
+  }
 }