You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/08/16 23:19:20 UTC

svn commit: r1514897 [1/2] - in /lucene/dev/branches/lucene5178: lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/ lucene/codecs/src/test/org/apache/lucene/codecs/diskdv/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core/src/java/org/apa...

Author: rmuir
Date: Fri Aug 16 21:19:19 2013
New Revision: 1514897

URL: http://svn.apache.org/r1514897
Log:
bump 4.5 codec

Added:
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskNormsFormat.java
      - copied, changed from r1514711, lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java   (with props)
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java
      - copied, changed from r1514711, lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java
      - copied, changed from r1514711, lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
      - copied, changed from r1514711, lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/package.html   (with props)
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/codecs/lucene45/
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/codecs/lucene45/TestLucene45DocValuesFormat.java
      - copied, changed from r1514711, lucene/dev/branches/lucene5178/lucene/codecs/src/test/org/apache/lucene/codecs/diskdv/TestDiskDocValuesFormat.java
Removed:
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
    lucene/dev/branches/lucene5178/lucene/codecs/src/test/org/apache/lucene/codecs/diskdv/TestCheapBastardDocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java
Modified:
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/package.html
    lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/lucene5178/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/branches/lucene5178/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldDocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/codecs/perfield/TestPerFieldPostingsFormat2.java
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
    lucene/dev/branches/lucene5178/lucene/core/src/test/org/apache/lucene/util/TestNamedSPILoader.java
    lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42Codec.java
    lucene/dev/branches/lucene5178/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesConsumer.java
    lucene/dev/branches/lucene5178/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/DoubleFieldSource.java
    lucene/dev/branches/lucene5178/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/FloatFieldSource.java
    lucene/dev/branches/lucene5178/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/IntFieldSource.java
    lucene/dev/branches/lucene5178/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/LongFieldSource.java
    lucene/dev/branches/lucene5178/lucene/site/xsl/index.xsl
    lucene/dev/branches/lucene5178/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
    lucene/dev/branches/lucene5178/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingNormsFormat.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/CompressingCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/BaseStoredFieldsFormatTestCase.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java
    lucene/dev/branches/lucene5178/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
    lucene/dev/branches/lucene5178/solr/core/src/java/org/apache/solr/core/SchemaCodecFactory.java
    lucene/dev/branches/lucene5178/solr/core/src/java/org/apache/solr/request/NumericFacets.java
    lucene/dev/branches/lucene5178/solr/core/src/test-files/solr/collection1/conf/schema_codec.xml
    lucene/dev/branches/lucene5178/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java

Modified: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java?rev=1514897&r1=1514896&r2=1514897&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java Fri Aug 16 21:19:19 2013
@@ -22,8 +22,11 @@ import java.io.IOException;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.BytesRef;
 
 /**
  * DocValues format that keeps most things on disk.
@@ -40,7 +43,12 @@ public final class DiskDocValuesFormat e
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    return new DiskDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+    return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION) {
+      @Override
+      protected void addTermsDict(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+        addBinaryField(field, values);
+      }
+    };
   }
 
   @Override
@@ -52,11 +60,4 @@ public final class DiskDocValuesFormat e
   public static final String DATA_EXTENSION = "dvdd";
   public static final String META_CODEC = "DiskDocValuesMetadata";
   public static final String META_EXTENSION = "dvdm";
-  public static final int VERSION_START = 0;
-  public static final int VERSION_COMPRESSED_TERMS = 1;
-  public static final int VERSION_CURRENT = VERSION_COMPRESSED_TERMS;
-  public static final byte NUMERIC = 0;
-  public static final byte BINARY = 1;
-  public static final byte SORTED = 2;
-  public static final byte SORTED_SET = 3;
 }

Modified: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java?rev=1514897&r1=1514896&r2=1514897&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java Fri Aug 16 21:19:19 2013
@@ -17,35 +17,26 @@ package org.apache.lucene.codecs.diskdv;
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.DELTA_COMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.GCD_COMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.TABLE_COMPRESSED;
-
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_PREFIX_COMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.TABLE_COMPRESSED;
 
 import java.io.IOException;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.DocValuesProducer.SortedSetDocsWithField;
+import org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer;
+import org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -56,32 +47,28 @@ import org.apache.lucene.util.packed.Pac
 
 class DiskDocValuesProducer extends DocValuesProducer {
   private final Map<Integer,NumericEntry> numerics;
-  private final Map<Integer,BinaryEntry> binaries;
   private final Map<Integer,NumericEntry> ords;
   private final Map<Integer,NumericEntry> ordIndexes;
+  private final Map<Integer,BinaryEntry> binaries;
   private final IndexInput data;
   private final int maxDoc;
-
-  // memory-resident structures
-  private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
-  private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
   
   DiskDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
+    this.maxDoc = state.segmentInfo.getDocCount();
     // read in the entries from the metadata file.
     IndexInput in = state.directory.openInput(metaName, state.context);
-    this.maxDoc = state.segmentInfo.getDocCount();
     boolean success = false;
     final int version;
     try {
       version = CodecUtil.checkHeader(in, metaCodec, 
-                                      DiskDocValuesFormat.VERSION_CURRENT,
-                                      DiskDocValuesFormat.VERSION_CURRENT);
+                                      Lucene45DocValuesFormat.VERSION_CURRENT,
+                                      Lucene45DocValuesFormat.VERSION_CURRENT);
       numerics = new HashMap<Integer,NumericEntry>();
       ords = new HashMap<Integer,NumericEntry>();
       ordIndexes = new HashMap<Integer,NumericEntry>();
       binaries = new HashMap<Integer,BinaryEntry>();
-      readFields(in, state.fieldInfos);
+      readFields(in);
 
       success = true;
     } finally {
@@ -97,10 +84,10 @@ class DiskDocValuesProducer extends DocV
       String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.openInput(dataName, state.context);
       final int version2 = CodecUtil.checkHeader(data, dataCodec, 
-                                                 DiskDocValuesFormat.VERSION_CURRENT,
-                                                 DiskDocValuesFormat.VERSION_CURRENT);
+                                                 Lucene45DocValuesFormat.VERSION_CURRENT,
+                                                 Lucene45DocValuesFormat.VERSION_CURRENT);
       if (version != version2) {
-        throw new CorruptIndexException("Format versions mismatch");
+        throw new CorruptIndexException("Versions mismatch");
       }
 
       success = true;
@@ -109,61 +96,62 @@ class DiskDocValuesProducer extends DocV
         IOUtils.closeWhileHandlingException(this.data);
       }
     }
+
   }
   
-  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
+  private void readFields(IndexInput meta) throws IOException {
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
       byte type = meta.readByte();
-      if (type == DiskDocValuesFormat.NUMERIC) {
+      if (type == Lucene45DocValuesFormat.NUMERIC) {
         numerics.put(fieldNumber, readNumericEntry(meta));
-      } else if (type == DiskDocValuesFormat.BINARY) {
+      } else if (type == Lucene45DocValuesFormat.BINARY) {
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
-      } else if (type == DiskDocValuesFormat.SORTED) {
+      } else if (type == Lucene45DocValuesFormat.SORTED) {
         // sorted = binary + numeric
         if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
         }
-        if (meta.readByte() != DiskDocValuesFormat.BINARY) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+        if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
+          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
         }
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
         
         if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
         }
-        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+        if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
+          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
         }
         NumericEntry n = readNumericEntry(meta);
         ords.put(fieldNumber, n);
-      } else if (type == DiskDocValuesFormat.SORTED_SET) {
+      } else if (type == Lucene45DocValuesFormat.SORTED_SET) {
         // sortedset = binary + numeric + ordIndex
         if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
         }
-        if (meta.readByte() != DiskDocValuesFormat.BINARY) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+        if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
         }
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
         
         if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
         }
-        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+        if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
         }
         NumericEntry n1 = readNumericEntry(meta);
         ords.put(fieldNumber, n1);
         
         if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
         }
-        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+        if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
         }
         NumericEntry n2 = readNumericEntry(meta);
         ordIndexes.put(fieldNumber, n2);
@@ -209,27 +197,18 @@ class DiskDocValuesProducer extends DocV
   
   static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
     BinaryEntry entry = new BinaryEntry();
-    entry.format = meta.readVInt();
+    int format = meta.readVInt();
+    if (format != Lucene45DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED && format != Lucene45DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED) {
+      throw new CorruptIndexException("Unexpected format for binary entry: " + format + ", input=" + meta);
+    }
     entry.minLength = meta.readVInt();
     entry.maxLength = meta.readVInt();
     entry.count = meta.readVLong();
     entry.offset = meta.readLong();
-    switch(entry.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        break;
-      case BINARY_PREFIX_COMPRESSED:
-        entry.addressInterval = meta.readVInt();
-        entry.addressesOffset = meta.readLong();
-        entry.packedIntsVersion = meta.readVInt();
-        entry.blockSize = meta.readVInt();
-        break;
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        entry.addressesOffset = meta.readLong();
-        entry.packedIntsVersion = meta.readVInt();
-        entry.blockSize = meta.readVInt();
-        break;
-      default:
-        throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
+    if (entry.minLength != entry.maxLength) {
+      entry.addressesOffset = meta.readLong();
+      entry.packedIntsVersion = meta.readVInt();
+      entry.blockSize = meta.readVInt();
     }
     return entry;
   }
@@ -237,10 +216,10 @@ class DiskDocValuesProducer extends DocV
   @Override
   public NumericDocValues getNumeric(FieldInfo field) throws IOException {
     NumericEntry entry = numerics.get(field.number);
-    return getNumeric(entry);
+    return getNumeric(field, entry);
   }
   
-  LongNumericDocValues getNumeric(NumericEntry entry) throws IOException {
+  private LongNumericDocValues getNumeric(FieldInfo field, final NumericEntry entry) throws IOException {
     final IndexInput data = this.data.clone();
     data.seek(entry.offset);
 
@@ -264,12 +243,12 @@ class DiskDocValuesProducer extends DocV
           }
         };
       case TABLE_COMPRESSED:
-        final long table[] = entry.table;
+        final long[] table = entry.table;
         final int bitsRequired = PackedInts.bitsRequired(table.length - 1);
         final PackedInts.Reader ords = PackedInts.getDirectReaderNoHeader(data, PackedInts.Format.PACKED, entry.packedIntsVersion, (int) entry.count, bitsRequired);
         return new LongNumericDocValues() {
           @Override
-          public long get(long id) {
+          long get(long id) {
             return table[(int) ords.get((int) id)];
           }
         };
@@ -281,15 +260,10 @@ class DiskDocValuesProducer extends DocV
   @Override
   public BinaryDocValues getBinary(FieldInfo field) throws IOException {
     BinaryEntry bytes = binaries.get(field.number);
-    switch(bytes.format) {
-      case BINARY_FIXED_UNCOMPRESSED:
-        return getFixedBinary(field, bytes);
-      case BINARY_VARIABLE_UNCOMPRESSED:
-        return getVariableBinary(field, bytes);
-      case BINARY_PREFIX_COMPRESSED:
-        return getCompressedBinary(field, bytes);
-      default:
-        throw new AssertionError();
+    if (bytes.minLength == bytes.maxLength) {
+      return getFixedBinary(field, bytes);
+    } else {
+      return getVariableBinary(field, bytes);
     }
   }
   
@@ -318,22 +292,13 @@ class DiskDocValuesProducer extends DocV
   
   private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
     final IndexInput data = this.data.clone();
-    
-    final MonotonicBlockPackedReader addresses;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
-        addressInstances.put(field.number, addrInstance);
-      }
-      addresses = addrInstance;
-    }
+    data.seek(bytes.addressesOffset);
 
+    final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, true);
     return new LongBinaryDocValues() {
       @Override
       public void get(long id, BytesRef result) {
-        long startAddress = bytes.offset + (id == 0 ? 0 : addresses.get(id-1));
+        long startAddress = bytes.offset + (id == 0 ? 0 : + addresses.get(id-1));
         long endAddress = bytes.offset + addresses.get(id);
         int length = (int) (endAddress - startAddress);
         try {
@@ -352,39 +317,11 @@ class DiskDocValuesProducer extends DocV
     };
   }
 
-  private BinaryDocValues getCompressedBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
-    final IndexInput data = this.data.clone();
-    final long interval = bytes.addressInterval;
-
-    final MonotonicBlockPackedReader addresses;
-    synchronized (addressInstances) {
-      MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
-      if (addrInstance == null) {
-        data.seek(bytes.addressesOffset);
-        final long size;
-        if (bytes.count % interval == 0) {
-          size = bytes.count / interval;
-        } else {
-          size = 1L + bytes.count / interval;
-        }
-        addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, size, false);
-        addressInstances.put(field.number, addrInstance);
-      }
-      addresses = addrInstance;
-    }
-    
-    return new CompressedBinaryDocValues(bytes, addresses, data);
-  }
-
   @Override
   public SortedDocValues getSorted(FieldInfo field) throws IOException {
     final int valueCount = (int) binaries.get(field.number).count;
     final BinaryDocValues binary = getBinary(field);
-    NumericEntry entry = ords.get(field.number);
-    IndexInput data = this.data.clone();
-    data.seek(entry.offset);
-    final BlockPackedReader ordinals = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
-    
+    final NumericDocValues ordinals = getNumeric(field, ords.get(field.number));
     return new SortedDocValues() {
 
       @Override
@@ -401,46 +338,18 @@ class DiskDocValuesProducer extends DocV
       public int getValueCount() {
         return valueCount;
       }
-
-      @Override
-      public int lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return (int) ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-        return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
     };
   }
 
   @Override
   public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
     final long valueCount = binaries.get(field.number).count;
-    // we keep the byte[]s and list of ords on disk, these could be large
     final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
-    final LongNumericDocValues ordinals = getNumeric(ords.get(field.number));
-    // but the addresses to the ord stream are in RAM
-    final MonotonicBlockPackedReader ordIndex;
-    synchronized (ordIndexInstances) {
-      MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
-      if (ordIndexInstance == null) {
-        NumericEntry entry = ordIndexes.get(field.number);
-        IndexInput data = this.data.clone();
-        data.seek(entry.offset);
-        ordIndexInstance = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
-        ordIndexInstances.put(field.number, ordIndexInstance);
-      }
-      ordIndex = ordIndexInstance;
-    }
+    final LongNumericDocValues ordinals = getNumeric(field, ords.get(field.number));
+    NumericEntry entry = ordIndexes.get(field.number);
+    IndexInput data = this.data.clone();
+    data.seek(entry.offset);
+    final MonotonicBlockPackedReader ordIndex = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
     
     return new SortedSetDocValues() {
       long offset;
@@ -472,31 +381,11 @@ class DiskDocValuesProducer extends DocV
       public long getValueCount() {
         return valueCount;
       }
-      
-      @Override
-      public long lookupTerm(BytesRef key) {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).lookupTerm(key);
-        } else {
-          return super.lookupTerm(key);
-        }
-      }
-
-      @Override
-      public TermsEnum termsEnum() {
-        if (binary instanceof CompressedBinaryDocValues) {
-          return ((CompressedBinaryDocValues)binary).getTermsEnum();
-        } else {
-          return super.termsEnum();
-        }
-      }
     };
   }
-
+  
   @Override
   public Bits getDocsWithField(FieldInfo field) throws IOException {
-    // nocommit: only use this if the field's entry has missing values (write that),
-    // otherwise return MatchAllBits
     if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
       return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
     } else {
@@ -525,12 +414,10 @@ class DiskDocValuesProducer extends DocV
   static class BinaryEntry {
     long offset;
 
-    int format;
     long count;
     int minLength;
     int maxLength;
     long addressesOffset;
-    long addressInterval;
     int packedIntsVersion;
     int blockSize;
   }
@@ -553,204 +440,4 @@ class DiskDocValuesProducer extends DocV
     
     abstract void get(long id, BytesRef Result);
   }
-  
-  // in the compressed case, we add a few additional operations for
-  // more efficient reverse lookup and enumeration
-  static class CompressedBinaryDocValues extends LongBinaryDocValues {
-    final BinaryEntry bytes;
-    final long interval;
-    final long numValues;
-    final long numIndexValues;
-    final MonotonicBlockPackedReader addresses;
-    final IndexInput data;
-    final TermsEnum termsEnum;
-    
-    public CompressedBinaryDocValues(BinaryEntry bytes, MonotonicBlockPackedReader addresses, IndexInput data) throws IOException {
-      this.bytes = bytes;
-      this.interval = bytes.addressInterval;
-      this.addresses = addresses;
-      this.data = data;
-      this.numValues = bytes.count;
-      this.numIndexValues = addresses.size();
-      this.termsEnum = getTermsEnum(data);
-    }
-    
-    @Override
-    public void get(long id, BytesRef result) {
-      try {
-        termsEnum.seekExact(id);
-        BytesRef term = termsEnum.term();
-        result.bytes = term.bytes;
-        result.offset = term.offset;
-        result.length = term.length;
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    
-    long lookupTerm(BytesRef key) {
-      try {
-        SeekStatus status = termsEnum.seekCeil(key);
-        if (status == SeekStatus.END) {
-          return -numValues-1;
-        } else if (status == SeekStatus.FOUND) {
-          return termsEnum.ord();
-        } else {
-          return -termsEnum.ord()-1;
-        }
-      } catch (IOException bogus) {
-        throw new RuntimeException(bogus);
-      }
-    }
-    
-    TermsEnum getTermsEnum() {
-      try {
-        return getTermsEnum(data.clone());
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    
-    private TermsEnum getTermsEnum(final IndexInput input) throws IOException {
-      input.seek(bytes.offset);
-      
-      return new TermsEnum() {
-        private long currentOrd = -1;
-        // TODO: maxLength is negative when all terms are merged away...
-        private final BytesRef termBuffer = new BytesRef(bytes.maxLength < 0 ? 0 : bytes.maxLength);
-        private final BytesRef term = new BytesRef(); // TODO: paranoia?
-
-        @Override
-        public BytesRef next() throws IOException {
-          if (doNext() == null) {
-            return null;
-          } else {
-            setTerm();
-            return term;
-          }
-        }
-        
-        private BytesRef doNext() throws IOException {
-          if (++currentOrd >= numValues) {
-            return null;
-          } else {
-            int start = input.readVInt();
-            int suffix = input.readVInt();
-            input.readBytes(termBuffer.bytes, start, suffix);
-            termBuffer.length = start + suffix;
-            return termBuffer;
-          }
-        }
-
-        @Override
-        public SeekStatus seekCeil(BytesRef text) throws IOException {
-          // binary-search just the index values to find the block,
-          // then scan within the block
-          long low = 0;
-          long high = numIndexValues-1;
-
-          while (low <= high) {
-            long mid = (low + high) >>> 1;
-            doSeek(mid * interval);
-            int cmp = termBuffer.compareTo(text);
-
-            if (cmp < 0) {
-              low = mid + 1;
-            } else if (cmp > 0) {
-              high = mid - 1;
-            } else {
-              // we got lucky, found an indexed term
-              setTerm();
-              return SeekStatus.FOUND;
-            }
-          }
-          
-          if (numIndexValues == 0) {
-            return SeekStatus.END;
-          }
-          
-          // block before insertion point
-          long block = low-1;
-          doSeek(block < 0 ? -1 : block * interval);
-          
-          while (doNext() != null) {
-            int cmp = termBuffer.compareTo(text);
-            if (cmp == 0) {
-              setTerm();
-              return SeekStatus.FOUND;
-            } else if (cmp > 0) {
-              setTerm();
-              return SeekStatus.NOT_FOUND;
-            }
-          }
-          
-          return SeekStatus.END;
-        }
-
-        @Override
-        public void seekExact(long ord) throws IOException {
-          doSeek(ord);
-          setTerm();
-        }
-        
-        private void doSeek(long ord) throws IOException {
-          long block = ord / interval;
-
-          if (ord >= currentOrd && block == currentOrd / interval) {
-            // seek within current block
-          } else {
-            // position before start of block
-            currentOrd = ord - ord % interval - 1;
-            input.seek(bytes.offset + addresses.get(block));
-          }
-          
-          while (currentOrd < ord) {
-            doNext();
-          }
-        }
-        
-        private void setTerm() {
-          // TODO: is there a cleaner way
-          term.bytes = new byte[termBuffer.length];
-          term.offset = 0;
-          term.copyBytes(termBuffer);
-        }
-
-        @Override
-        public BytesRef term() throws IOException {
-          return term;
-        }
-
-        @Override
-        public long ord() throws IOException {
-          return currentOrd;
-        }
-        
-        @Override
-        public Comparator<BytesRef> getComparator() {
-          return BytesRef.getUTF8SortedAsUnicodeComparator();
-        }
-
-        @Override
-        public int docFreq() throws IOException {
-          throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public long totalTermFreq() throws IOException {
-          return -1;
-        }
-
-        @Override
-        public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-          throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-          throw new UnsupportedOperationException();
-        }
-      };
-    }
-  }
 }

Copied: lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskNormsFormat.java (from r1514711, lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskNormsFormat.java?p2=lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskNormsFormat.java&p1=lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java&r1=1514711&r2=1514897&rev=1514897&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java (original)
+++ lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskNormsFormat.java Fri Aug 16 21:19:19 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.cheapbastard;
+package org.apache.lucene.codecs.diskdv;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -22,25 +22,25 @@ import java.io.IOException;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.NormsFormat;
-import org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer;
+import org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 
 /** Norms format that keeps all norms on disk */
-public final class CheapBastardNormsFormat extends NormsFormat {
+public final class DiskNormsFormat extends NormsFormat {
 
   @Override
   public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
-    return new DiskDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+    return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
   }
 
   @Override
   public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
-    return new CheapBastardDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+    return new DiskDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
   }
   
-  static final String DATA_CODEC = "CheapBastardNormsData";
-  static final String DATA_EXTENSION = "cbnd";
-  static final String META_CODEC = "CheapBastardNormsMetadata";
-  static final String META_EXTENSION = "cbnm";
+  static final String DATA_CODEC = "DiskNormsData";
+  static final String DATA_EXTENSION = "dnvd";
+  static final String META_CODEC = "DiskNormsMetadata";
+  static final String META_EXTENSION = "dnvm";
 }

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/Codec.java?rev=1514897&r1=1514896&r2=1514897&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/Codec.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/Codec.java Fri Aug 16 21:19:19 2013
@@ -119,7 +119,7 @@ public abstract class Codec implements N
     loader.reload(classloader);
   }
   
-  private static Codec defaultCodec = Codec.forName("Lucene42");
+  private static Codec defaultCodec = Codec.forName("Lucene45");
   
   /** expert: returns the default codec used for newly created
    *  {@link IndexWriterConfig}s.

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java?rev=1514897&r1=1514896&r2=1514897&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/FilterCodec.java Fri Aug 16 21:19:19 2013
@@ -21,13 +21,13 @@ package org.apache.lucene.codecs;
  * A codec that forwards all its method calls to another codec.
  * <p>
  * Extend this class when you need to reuse the functionality of an existing
- * codec. For example, if you want to build a codec that redefines Lucene42's
+ * codec. For example, if you want to build a codec that redefines Lucene45's
  * {@link LiveDocsFormat}:
  * <pre class="prettyprint">
  *   public final class CustomCodec extends FilterCodec {
  *
  *     public CustomCodec() {
- *       super("CustomCodec", new Lucene42Codec());
+ *       super("CustomCodec", new Lucene45Codec());
  *     }
  *
  *     public LiveDocsFormat liveDocsFormat() {

Modified: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java?rev=1514897&r1=1514896&r2=1514897&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java Fri Aug 16 21:19:19 2013
@@ -27,7 +27,6 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
 
 /**

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

Added: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java?rev=1514897&view=auto
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java (added)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45Codec.java Fri Aug 16 21:19:19 2013
@@ -0,0 +1,141 @@
+package org.apache.lucene.codecs.lucene45;
+
+/*
+ * 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.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.NormsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+/**
+ * Implements the Lucene 4.5 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.lucene45 package documentation for file format details.
+ * @lucene.experimental
+ */
+// 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).
+public class Lucene45Codec extends Codec {
+  private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
+  private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
+  private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat();
+  private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
+  private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
+  
+  private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
+    @Override
+    public PostingsFormat getPostingsFormatForField(String field) {
+      return Lucene45Codec.this.getPostingsFormatForField(field);
+    }
+  };
+  
+  
+  private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
+    @Override
+    public DocValuesFormat getDocValuesFormatForField(String field) {
+      return Lucene45Codec.this.getDocValuesFormatForField(field);
+    }
+  };
+
+  /** Sole constructor. */
+  public Lucene45Codec() {
+    super("Lucene45");
+  }
+  
+  @Override
+  public final StoredFieldsFormat storedFieldsFormat() {
+    return fieldsFormat;
+  }
+  
+  @Override
+  public final TermVectorsFormat termVectorsFormat() {
+    return vectorsFormat;
+  }
+
+  @Override
+  public final PostingsFormat postingsFormat() {
+    return postingsFormat;
+  }
+  
+  @Override
+  public final FieldInfosFormat fieldInfosFormat() {
+    return fieldInfosFormat;
+  }
+  
+  @Override
+  public final SegmentInfoFormat segmentInfoFormat() {
+    return infosFormat;
+  }
+  
+  @Override
+  public final LiveDocsFormat liveDocsFormat() {
+    return liveDocsFormat;
+  }
+
+  /** Returns the postings format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene41"
+   */
+  public PostingsFormat getPostingsFormatForField(String field) {
+    return defaultFormat;
+  }
+  
+  /** Returns the docvalues format that should be used for writing 
+   *  new segments of <code>field</code>.
+   *  
+   *  The default implementation always returns "Lucene45"
+   */
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    return defaultDVFormat;
+  }
+  
+  @Override
+  public final DocValuesFormat docValuesFormat() {
+    return docValuesFormat;
+  }
+
+  private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
+  private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene45");
+
+  private final NormsFormat normsFormat = new Lucene42NormsFormat();
+
+  @Override
+  public final NormsFormat normsFormat() {
+    return normsFormat;
+  }
+}

Copied: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java (from r1514711, lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java?p2=lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java&p1=lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java&r1=1514711&r2=1514897&rev=1514897&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java Fri Aug 16 21:19:19 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.diskdv;
+package org.apache.lucene.codecs.lucene45;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -37,8 +37,8 @@ import org.apache.lucene.util.packed.Blo
 import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
 import org.apache.lucene.util.packed.PackedInts;
 
-/** writer for {@link DiskDocValuesFormat} */
-public class DiskDocValuesConsumer extends DocValuesConsumer {
+/** writer for {@link Lucene45DocValuesFormat} */
+public class Lucene45DocValuesConsumer extends DocValuesConsumer {
 
   static final int BLOCK_SIZE = 16384;
   static final int ADDRESS_INTERVAL = 16;
@@ -60,15 +60,15 @@ public class DiskDocValuesConsumer exten
   final IndexOutput data, meta;
   final int maxDoc;
   
-  public DiskDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+  public Lucene45DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     boolean success = false;
     try {
       String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.createOutput(dataName, state.context);
-      CodecUtil.writeHeader(data, dataCodec, DiskDocValuesFormat.VERSION_CURRENT);
+      CodecUtil.writeHeader(data, dataCodec, Lucene45DocValuesFormat.VERSION_CURRENT);
       String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
       meta = state.directory.createOutput(metaName, state.context);
-      CodecUtil.writeHeader(meta, metaCodec, DiskDocValuesFormat.VERSION_CURRENT);
+      CodecUtil.writeHeader(meta, metaCodec, Lucene45DocValuesFormat.VERSION_CURRENT);
       maxDoc = state.segmentInfo.getDocCount();
       success = true;
     } finally {
@@ -140,7 +140,7 @@ public class DiskDocValuesConsumer exten
       format = DELTA_COMPRESSED;
     }
     meta.writeVInt(field.number);
-    meta.writeByte(DiskDocValuesFormat.NUMERIC);
+    meta.writeByte(Lucene45DocValuesFormat.NUMERIC);
     meta.writeVInt(format);
     meta.writeVInt(PackedInts.VERSION_CURRENT);
     meta.writeLong(data.getFilePointer());
@@ -189,7 +189,7 @@ public class DiskDocValuesConsumer exten
   public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
     // write the byte[] data
     meta.writeVInt(field.number);
-    meta.writeByte(DiskDocValuesFormat.BINARY);
+    meta.writeByte(Lucene45DocValuesFormat.BINARY);
     int minLength = Integer.MAX_VALUE;
     int maxLength = Integer.MIN_VALUE;
     final long startFP = data.getFilePointer();
@@ -242,7 +242,7 @@ public class DiskDocValuesConsumer exten
     } else {
       // header
       meta.writeVInt(field.number);
-      meta.writeByte(DiskDocValuesFormat.BINARY);
+      meta.writeByte(Lucene45DocValuesFormat.BINARY);
       meta.writeVInt(BINARY_PREFIX_COMPRESSED);
       // now write the bytes: sharing prefixes within a block
       final long startFP = data.getFilePointer();
@@ -315,7 +315,7 @@ public class DiskDocValuesConsumer exten
       values = MissingOrdRemapper.insertEmptyValue(values);
     }
     meta.writeVInt(field.number);
-    meta.writeByte(DiskDocValuesFormat.SORTED);
+    meta.writeByte(Lucene45DocValuesFormat.SORTED);
     addTermsDict(field, values);
     addNumericField(field, docToOrd, false);
   }
@@ -323,7 +323,7 @@ public class DiskDocValuesConsumer exten
   @Override
   public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
     meta.writeVInt(field.number);
-    meta.writeByte(DiskDocValuesFormat.SORTED_SET);
+    meta.writeByte(Lucene45DocValuesFormat.SORTED_SET);
     // write the ord -> byte[] as a binary field
     addTermsDict(field, values);
     // write the stream of ords as a numeric field
@@ -332,7 +332,7 @@ public class DiskDocValuesConsumer exten
     
     // write the doc -> ord count as a absolute index to the stream
     meta.writeVInt(field.number);
-    meta.writeByte(DiskDocValuesFormat.NUMERIC);
+    meta.writeByte(Lucene45DocValuesFormat.NUMERIC);
     meta.writeVInt(DELTA_COMPRESSED);
     meta.writeVInt(PackedInts.VERSION_CURRENT);
     meta.writeLong(data.getFilePointer());

Copied: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java (from r1514711, lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java?p2=lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java&p1=lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java&r1=1514711&r2=1514897&rev=1514897&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java Fri Aug 16 21:19:19 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.diskdv;
+package org.apache.lucene.codecs.lucene45;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -19,42 +19,147 @@ package org.apache.lucene.codecs.diskdv;
 
 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;
 
 /**
- * DocValues format that keeps most things on disk.
+ * Lucene 4.5 DocValues format.
  * <p>
- * Only things like disk offsets are loaded into ram.
+ * 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: nocommit
+ * </ul>
+ * <p>
+ * {@link DocValuesType#SORTED SORTED}:
+ * <ul>
+ *    <li>Sorted: an FST mapping deduplicated terms to ordinals is written, 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: an FST mapping deduplicated terms to ordinals is written, along with the per-document
+ *        ordinal list written using one of the binary 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;FieldNumber,EntryType,Entry&gt;<sup>NumFields</sup></p>
+ *   <ul>
+ *     <li>Entry --&gt; NumericEntry | BinaryEntry | SortedEntry</li>
+ *     <li>NumericEntry --&gt; DataOffset,NumericCompressionType,PackedVersion</li>
+ *     <li>BinaryEntry --&gt; DataOffset,DataLength,MinLength,MaxLength,PackedVersion?,BlockSize?</li>
+ *     <li>SortedEntry --&gt; DataOffset,ValueCount</li>
+ *     <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *     <li>DataOffset,DataLength --&gt; {@link DataOutput#writeLong Int64}</li>
+ *     <li>EntryType,CompressionType --&gt; {@link DataOutput#writeByte Byte}</li>
+ *     <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *   </ul>
+ *   <p>Sorted fields have two entries: a SortedEntry with the FST metadata,
+ *      and an ordinary NumericEntry for the document-to-ord metadata.</p>
+ *   <p>SortedSet fields have two entries: a SortedEntry with the FST metadata,
+ *      and an ordinary BinaryEntry for the document-to-ord-list metadata.</p>
+ *   <p>FieldNumber of -1 indicates the end of metadata.</p>
+ *   <p>EntryType is a 0 (NumericEntry), 1 (BinaryEntry, or 2 (SortedEntry)</p>
+ *   <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
+ *   <p>NumericCompressionType 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>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.
+ *   <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></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; TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,{@link PackedInts PackedInts}</li>
+ *     <li>GCDCompressedNumerics --&gt; MinValue,GCD,{@link BlockPackedWriter BlockPackedInts(blockSize=16k)}</li>
+ *     <li>Addresses --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=16k)}</li>
+ *     <li>TableSize --&gt; {@link DataOutput#writeVInt vInt}</li>
+ *     <li>MinValue --&gt; {@link DataOutput#writeLong Int64}</li>
+ *     <li>GCD --&gt; {@link DataOutput#writeLong Int64}</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>
  * @lucene.experimental
  */
-public final class DiskDocValuesFormat extends DocValuesFormat {
+// nocommit: docs are incomplete
+public final class Lucene45DocValuesFormat extends DocValuesFormat {
 
-  public DiskDocValuesFormat() {
-    super("Disk");
+  public Lucene45DocValuesFormat() {
+    super("Lucene45");
   }
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    return new DiskDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+    return new Lucene45DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
   }
 
   @Override
   public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
-    return new DiskDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
+    return new Lucene45DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
   }
   
-  public static final String DATA_CODEC = "DiskDocValuesData";
-  public static final String DATA_EXTENSION = "dvdd";
-  public static final String META_CODEC = "DiskDocValuesMetadata";
-  public static final String META_EXTENSION = "dvdm";
+  public static final String DATA_CODEC = "Lucene45DocValuesData";
+  public static final String DATA_EXTENSION = "dvd";
+  public static final String META_CODEC = "Lucene45ValuesMetadata";
+  public static final String META_EXTENSION = "dvm";
   public static final int VERSION_START = 0;
-  public static final int VERSION_COMPRESSED_TERMS = 1;
-  public static final int VERSION_CURRENT = VERSION_COMPRESSED_TERMS;
+  public static final int VERSION_CURRENT = VERSION_START;
   public static final byte NUMERIC = 0;
   public static final byte BINARY = 1;
   public static final byte SORTED = 2;

Copied: lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java (from r1514711, lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java?p2=lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java&p1=lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java&r1=1514711&r2=1514897&rev=1514897&view=diff
==============================================================================
--- lucene/dev/branches/lucene5178/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene5178/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java Fri Aug 16 21:19:19 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.diskdv;
+package org.apache.lucene.codecs.lucene45;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,13 +17,13 @@ package org.apache.lucene.codecs.diskdv;
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.DELTA_COMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.GCD_COMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.TABLE_COMPRESSED;
-
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
-import static org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer.BINARY_PREFIX_COMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.TABLE_COMPRESSED;
+
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
 
 import java.io.IOException;
 import java.util.Comparator;
@@ -32,7 +32,6 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesProducer;
-import org.apache.lucene.codecs.DocValuesProducer.SortedSetDocsWithField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsAndPositionsEnum;
@@ -54,7 +53,7 @@ import org.apache.lucene.util.packed.Blo
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
 import org.apache.lucene.util.packed.PackedInts;
 
-class DiskDocValuesProducer extends DocValuesProducer {
+class Lucene45DocValuesProducer extends DocValuesProducer {
   private final Map<Integer,NumericEntry> numerics;
   private final Map<Integer,BinaryEntry> binaries;
   private final Map<Integer,NumericEntry> ords;
@@ -66,7 +65,7 @@ class DiskDocValuesProducer extends DocV
   private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
   private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
   
-  DiskDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
+  Lucene45DocValuesProducer(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.
     IndexInput in = state.directory.openInput(metaName, state.context);
@@ -75,8 +74,8 @@ class DiskDocValuesProducer extends DocV
     final int version;
     try {
       version = CodecUtil.checkHeader(in, metaCodec, 
-                                      DiskDocValuesFormat.VERSION_CURRENT,
-                                      DiskDocValuesFormat.VERSION_CURRENT);
+                                      Lucene45DocValuesFormat.VERSION_CURRENT,
+                                      Lucene45DocValuesFormat.VERSION_CURRENT);
       numerics = new HashMap<Integer,NumericEntry>();
       ords = new HashMap<Integer,NumericEntry>();
       ordIndexes = new HashMap<Integer,NumericEntry>();
@@ -97,8 +96,8 @@ class DiskDocValuesProducer extends DocV
       String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.openInput(dataName, state.context);
       final int version2 = CodecUtil.checkHeader(data, dataCodec, 
-                                                 DiskDocValuesFormat.VERSION_CURRENT,
-                                                 DiskDocValuesFormat.VERSION_CURRENT);
+                                                 Lucene45DocValuesFormat.VERSION_CURRENT,
+                                                 Lucene45DocValuesFormat.VERSION_CURRENT);
       if (version != version2) {
         throw new CorruptIndexException("Format versions mismatch");
       }
@@ -115,17 +114,17 @@ class DiskDocValuesProducer extends DocV
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
       byte type = meta.readByte();
-      if (type == DiskDocValuesFormat.NUMERIC) {
+      if (type == Lucene45DocValuesFormat.NUMERIC) {
         numerics.put(fieldNumber, readNumericEntry(meta));
-      } else if (type == DiskDocValuesFormat.BINARY) {
+      } else if (type == Lucene45DocValuesFormat.BINARY) {
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
-      } else if (type == DiskDocValuesFormat.SORTED) {
+      } else if (type == Lucene45DocValuesFormat.SORTED) {
         // sorted = binary + numeric
         if (meta.readVInt() != fieldNumber) {
           throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
-        if (meta.readByte() != DiskDocValuesFormat.BINARY) {
+        if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
           throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
         BinaryEntry b = readBinaryEntry(meta);
@@ -134,17 +133,17 @@ class DiskDocValuesProducer extends DocV
         if (meta.readVInt() != fieldNumber) {
           throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
-        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
+        if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
           throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
         NumericEntry n = readNumericEntry(meta);
         ords.put(fieldNumber, n);
-      } else if (type == DiskDocValuesFormat.SORTED_SET) {
+      } else if (type == Lucene45DocValuesFormat.SORTED_SET) {
         // sortedset = binary + numeric + ordIndex
         if (meta.readVInt() != fieldNumber) {
           throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
-        if (meta.readByte() != DiskDocValuesFormat.BINARY) {
+        if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
           throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
         BinaryEntry b = readBinaryEntry(meta);
@@ -153,7 +152,7 @@ class DiskDocValuesProducer extends DocV
         if (meta.readVInt() != fieldNumber) {
           throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
-        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
+        if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
           throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
         NumericEntry n1 = readNumericEntry(meta);
@@ -162,7 +161,7 @@ class DiskDocValuesProducer extends DocV
         if (meta.readVInt() != fieldNumber) {
           throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
-        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
+        if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
           throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
         }
         NumericEntry n2 = readNumericEntry(meta);