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/01/31 21:12:58 UTC

svn commit: r1441171 - in /lucene/dev/branches/lucene4547/lucene: codecs/src/java/org/apache/lucene/codecs/diskdv/ suggest/src/java/org/apache/lucene/search/suggest/ test-framework/src/java/org/apache/lucene/codecs/cheapbastard/ test-framework/src/java...

Author: rmuir
Date: Thu Jan 31 20:12:57 2013
New Revision: 1441171

URL: http://svn.apache.org/viewvc?rev=1441171&view=rev
Log:
fix Disk to be reasonable, keep unreasonable version in test-framework

Added:
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java   (with props)
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java
      - copied, changed from r1440942, lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java
      - copied, changed from r1440942, lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java   (with props)
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/package.html   (with props)
Modified:
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
    lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
    lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java?rev=1441171&r1=1441170&r2=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java Thu Jan 31 20:12:57 2013
@@ -31,24 +31,23 @@ import org.apache.lucene.util.packed.Blo
 import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
 import org.apache.lucene.util.packed.PackedInts;
 
-class DiskDocValuesConsumer extends DocValuesConsumer {
+/** writer for {@link DiskDocValuesFormat} */
+public class DiskDocValuesConsumer extends DocValuesConsumer {
 
   static final int BLOCK_SIZE = 16384;
 
   final IndexOutput data, meta;
   final int maxDoc;
   
-  DiskDocValuesConsumer(SegmentWriteState state) throws IOException {
+  public DiskDocValuesConsumer(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, "ddvd");
+      String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.createOutput(dataName, state.context);
-      CodecUtil.writeHeader(data, DiskDocValuesFormat.DATA_CODEC, 
-                                  DiskDocValuesFormat.VERSION_CURRENT);
-      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, "ddvm");
+      CodecUtil.writeHeader(data, dataCodec, DiskDocValuesFormat.VERSION_CURRENT);
+      String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
       meta = state.directory.createOutput(metaName, state.context);
-      CodecUtil.writeHeader(meta, DiskDocValuesFormat.METADATA_CODEC, 
-                                  DiskDocValuesFormat.VERSION_CURRENT);
+      CodecUtil.writeHeader(meta, metaCodec, DiskDocValuesFormat.VERSION_CURRENT);
       maxDoc = state.segmentInfo.getDocCount();
       success = true;
     } finally {
@@ -66,6 +65,7 @@ class DiskDocValuesConsumer extends DocV
     }
 
     meta.writeVInt(field.number);
+    meta.writeByte(DiskDocValuesFormat.NUMERIC);
     meta.writeVInt(PackedInts.VERSION_CURRENT);
     meta.writeLong(data.getFilePointer());
     meta.writeVInt(count);
@@ -82,6 +82,7 @@ class DiskDocValuesConsumer extends DocV
   public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
     // write the byte[] data
     meta.writeVInt(field.number);
+    meta.writeByte(DiskDocValuesFormat.BINARY);
     int minLength = Integer.MAX_VALUE;
     int maxLength = Integer.MIN_VALUE;
     final long startFP = data.getFilePointer();
@@ -116,6 +117,8 @@ class DiskDocValuesConsumer extends DocV
 
   @Override
   public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(DiskDocValuesFormat.SORTED);
     addBinaryField(field, values);
     addNumericField(field, docToOrd);
   }

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java?rev=1441171&r1=1441170&r2=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java Thu Jan 31 20:12:57 2013
@@ -26,17 +26,10 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.SegmentWriteState;
 
 /**
- * DocValues format that keeps everything on disk.
+ * DocValues format that keeps most things on disk.
  * <p>
- * Internally there are only 2 field types:
- * <ul>
- *   <li>BINARY: a big byte[].
- *   <li>NUMERIC: packed ints
- * </ul>
- * SORTED is encoded as BINARY + NUMERIC
- * <p>
- * NOTE: Don't use this format in production (its not very efficient).
- * Most likely you would want some parts in RAM, other parts on disk. 
+ * Things like ordinals and disk offsets are loaded into ram,
+ * for single-seek access to all the types.
  * <p>
  * @lucene.experimental
  */
@@ -48,16 +41,21 @@ public final class DiskDocValuesFormat e
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    return new DiskDocValuesConsumer(state);
+    return new DiskDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
   }
 
   @Override
   public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
-    return new DiskDocValuesProducer(state);
+    return new DiskDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
   }
   
-  static final String DATA_CODEC = "DiskDocValuesData";
-  static final String METADATA_CODEC = "DiskDocValuesMetadata";
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
+  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 int VERSION_START = 0;
+  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;
 }

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java?rev=1441171&r1=1441170&r2=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java Thu Jan 31 20:12:57 2013
@@ -26,7 +26,6 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.NumericDocValues;
@@ -40,17 +39,21 @@ import org.apache.lucene.util.packed.Mon
 
 class DiskDocValuesProducer extends DocValuesProducer {
   private final Map<Integer,NumericEntry> numerics;
-  private final Map<Integer,NumericEntry> ords;
   private final Map<Integer,BinaryEntry> binaries;
+  private final Map<Integer,NumericEntry> ords;
   private final IndexInput data;
+
+  // memory-resident structures
+  private final Map<Integer,BlockPackedReader> ordinalInstances = new HashMap<Integer,BlockPackedReader>();
+  private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
   
-  DiskDocValuesProducer(SegmentReadState state) throws IOException {
-    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, "ddvm");
+  DiskDocValuesProducer(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);
     boolean success = false;
     try {
-      CodecUtil.checkHeader(in, DiskDocValuesFormat.METADATA_CODEC, 
+      CodecUtil.checkHeader(in, metaCodec, 
                                 DiskDocValuesFormat.VERSION_START,
                                 DiskDocValuesFormat.VERSION_START);
       numerics = new HashMap<Integer,NumericEntry>();
@@ -66,9 +69,9 @@ class DiskDocValuesProducer extends DocV
       }
     }
     
-    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, "ddvd");
+    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
     data = state.directory.openInput(dataName, state.context);
-    CodecUtil.checkHeader(data, DiskDocValuesFormat.DATA_CODEC, 
+    CodecUtil.checkHeader(data, dataCodec, 
                                 DiskDocValuesFormat.VERSION_START,
                                 DiskDocValuesFormat.VERSION_START);
   }
@@ -76,18 +79,29 @@ class DiskDocValuesProducer extends DocV
   private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
-      DocValuesType type = infos.fieldInfo(fieldNumber).getDocValuesType();
-      if (type == DocValuesType.NUMERIC) {
+      byte type = meta.readByte();
+      if (type == DiskDocValuesFormat.NUMERIC) {
         numerics.put(fieldNumber, readNumericEntry(meta));
-      } else if (type == DocValuesType.BINARY) {
+      } else if (type == DiskDocValuesFormat.BINARY) {
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
-      } else if (type == DocValuesType.SORTED) {
+      } else if (type == DiskDocValuesFormat.SORTED) {
+        // sorted = binary + numeric
+        if (meta.readVInt() != fieldNumber) {
+          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");
+        }
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
+        
         if (meta.readVInt() != fieldNumber) {
           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");
+        }
         NumericEntry n = readNumericEntry(meta);
         ords.put(fieldNumber, n);
       }
@@ -121,10 +135,6 @@ class DiskDocValuesProducer extends DocV
   @Override
   public NumericDocValues getNumeric(FieldInfo field) throws IOException {
     NumericEntry entry = numerics.get(field.number);
-    return getNumeric(field, entry);
-  }
-  
-  private NumericDocValues getNumeric(FieldInfo field, final NumericEntry entry) throws IOException {
     final IndexInput data = this.data.clone();
     data.seek(entry.offset);
 
@@ -172,9 +182,18 @@ class DiskDocValuesProducer extends DocV
   
   private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
     final IndexInput data = this.data.clone();
-    data.seek(bytes.addressesOffset);
+    
+    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;
+    }
 
-    final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, true);
     return new BinaryDocValues() {
       @Override
       public void get(int docID, BytesRef result) {
@@ -201,7 +220,18 @@ class DiskDocValuesProducer extends DocV
   public SortedDocValues getSorted(FieldInfo field) throws IOException {
     final int valueCount = binaries.get(field.number).count;
     final BinaryDocValues binary = getBinary(field);
-    final NumericDocValues ordinals = getNumeric(field, ords.get(field.number));
+    final BlockPackedReader ordinals;
+    synchronized (ordinalInstances) {
+      BlockPackedReader ordsInstance = ordinalInstances.get(field.number);
+      if (ordsInstance == null) {
+        NumericEntry entry = ords.get(field.number);
+        IndexInput data = this.data.clone();
+        data.seek(entry.offset);
+        ordsInstance = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
+        ordinalInstances.put(field.number, ordsInstance);
+      }
+      ordinals = ordsInstance;
+    }
     return new SortedDocValues() {
 
       @Override

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java?rev=1441171&r1=1441170&r2=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java Thu Jan 31 20:12:57 2013
@@ -38,7 +38,7 @@ import org.apache.lucene.util.SorterTemp
  * @lucene.internal
  * @lucene.experimental
  */
-final class BytesRefArray {
+public final class BytesRefArray {
   private final ByteBlockPool pool;
   private int[] offsets = new int[1];
   private int lastElement = 0;

Added: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java?rev=1441171&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java (added)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardCodec.java Thu Jan 31 20:12:57 2013
@@ -0,0 +1,72 @@
+package org.apache.lucene.codecs.cheapbastard;
+
+/*
+ * 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.DocValuesFormat;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42Codec;
+
+/** Codec that tries to use as little ram as possible because he spent all his money on beer */
+// TODO: better name :) 
+// but if we named it "LowMemory" in codecs/ package, it would be irresistible like optimize()!
+public class CheapBastardCodec extends FilterCodec {
+  
+  // TODO: would be better to have no terms index at all and bsearch a terms dict
+  private final PostingsFormat postings = new Lucene41PostingsFormat(100, 200);
+  // uncompressing versions, waste lots of disk but no ram
+  private final StoredFieldsFormat storedFields = new Lucene40StoredFieldsFormat();
+  private final TermVectorsFormat termVectors = new Lucene40TermVectorsFormat();
+  // these go to disk for all docvalues/norms datastructures
+  private final DocValuesFormat docValues = new CheapBastardDocValuesFormat();
+  private final NormsFormat norms = new CheapBastardNormsFormat();
+
+  public CheapBastardCodec() {
+    super("CheapBastard", new Lucene42Codec());
+  }
+  
+  public PostingsFormat postingsFormat() {
+    return postings;
+  }
+
+  @Override
+  public DocValuesFormat docValuesFormat() {
+    return docValues;
+  }
+  
+  @Override
+  public NormsFormat normsFormat() {
+    return norms;
+  }
+  
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    return storedFields;
+  }
+
+  @Override
+  public TermVectorsFormat termVectorsFormat() {
+    return termVectors;
+  }
+}

Copied: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java (from r1440942, lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java?p2=lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java&p1=lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java&r1=1440942&r2=1441171&rev=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java Thu Jan 31 20:12:57 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.diskdv;
+package org.apache.lucene.codecs.cheapbastard;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -22,6 +22,8 @@ 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.diskdv.DiskDocValuesConsumer;
+import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 
@@ -40,24 +42,25 @@ import org.apache.lucene.index.SegmentWr
  * <p>
  * @lucene.experimental
  */
-public final class DiskDocValuesFormat extends DocValuesFormat {
+public final class CheapBastardDocValuesFormat extends DocValuesFormat {
 
-  public DiskDocValuesFormat() {
-    super("Disk");
+  public CheapBastardDocValuesFormat() {
+    super("CheapBastard");
   }
 
   @Override
   public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    return new DiskDocValuesConsumer(state);
+    return new DiskDocValuesConsumer(state, DiskDocValuesFormat.DATA_CODEC, 
+                                            DiskDocValuesFormat.DATA_EXTENSION, 
+                                            DiskDocValuesFormat.META_CODEC, 
+                                            DiskDocValuesFormat.META_EXTENSION);
   }
 
   @Override
   public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
-    return new DiskDocValuesProducer(state);
+    return new CheapBastardDocValuesProducer(state, DiskDocValuesFormat.DATA_CODEC, 
+                                                    DiskDocValuesFormat.DATA_EXTENSION, 
+                                                    DiskDocValuesFormat.META_CODEC, 
+                                                    DiskDocValuesFormat.META_EXTENSION);
   }
-  
-  static final String DATA_CODEC = "DiskDocValuesData";
-  static final String METADATA_CODEC = "DiskDocValuesMetadata";
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
 }

Copied: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java (from r1440942, lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java?p2=lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java&p1=lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java&r1=1440942&r2=1441171&rev=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java Thu Jan 31 20:12:57 2013
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.diskdv;
+package org.apache.lucene.codecs.cheapbastard;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -23,11 +23,10 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
-import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
@@ -38,25 +37,25 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.BlockPackedReader;
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
 
-class DiskDocValuesProducer extends DocValuesProducer {
+class CheapBastardDocValuesProducer extends DocValuesProducer {
   private final Map<Integer,NumericEntry> numerics;
   private final Map<Integer,NumericEntry> ords;
   private final Map<Integer,BinaryEntry> binaries;
   private final IndexInput data;
   
-  DiskDocValuesProducer(SegmentReadState state) throws IOException {
-    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, "ddvm");
+  CheapBastardDocValuesProducer(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);
     boolean success = false;
     try {
-      CodecUtil.checkHeader(in, DiskDocValuesFormat.METADATA_CODEC, 
+      CodecUtil.checkHeader(in, metaCodec, 
                                 DiskDocValuesFormat.VERSION_START,
                                 DiskDocValuesFormat.VERSION_START);
       numerics = new HashMap<Integer,NumericEntry>();
       ords = new HashMap<Integer,NumericEntry>();
       binaries = new HashMap<Integer,BinaryEntry>();
-      readFields(in, state.fieldInfos);
+      readFields(in);
       success = true;
     } finally {
       if (success) {
@@ -66,28 +65,39 @@ class DiskDocValuesProducer extends DocV
       }
     }
     
-    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, "ddvd");
+    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
     data = state.directory.openInput(dataName, state.context);
-    CodecUtil.checkHeader(data, DiskDocValuesFormat.DATA_CODEC, 
+    CodecUtil.checkHeader(data, dataCodec, 
                                 DiskDocValuesFormat.VERSION_START,
                                 DiskDocValuesFormat.VERSION_START);
   }
   
-  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
+  private void readFields(IndexInput meta) throws IOException {
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
-      DocValuesType type = infos.fieldInfo(fieldNumber).getDocValuesType();
-      if (type == DocValuesType.NUMERIC) {
+      byte type = meta.readByte();
+      if (type == DiskDocValuesFormat.NUMERIC) {
         numerics.put(fieldNumber, readNumericEntry(meta));
-      } else if (type == DocValuesType.BINARY) {
+      } else if (type == DiskDocValuesFormat.BINARY) {
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
-      } else if (type == DocValuesType.SORTED) {
+      } else if (type == DiskDocValuesFormat.SORTED) {
+        // sorted = binary + numeric
+        if (meta.readVInt() != fieldNumber) {
+          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");
+        }
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
+        
         if (meta.readVInt() != fieldNumber) {
           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");
+        }
         NumericEntry n = readNumericEntry(meta);
         ords.put(fieldNumber, n);
       }

Added: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java?rev=1441171&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java (added)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardNormsFormat.java Thu Jan 31 20:12:57 2013
@@ -0,0 +1,46 @@
+package org.apache.lucene.codecs.cheapbastard;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer;
+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 {
+
+  @Override
+  public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
+    return new DiskDocValuesConsumer(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);
+  }
+  
+  static final String DATA_CODEC = "CheapBastardNormsData";
+  static final String DATA_EXTENSION = "cbnd";
+  static final String META_CODEC = "CheapBastardNormsMetadata";
+  static final String META_EXTENSION = "cbnm";
+}

Added: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/package.html?rev=1441171&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/package.html (added)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/package.html Thu Jan 31 20:12:57 2013
@@ -0,0 +1,26 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Codec that unreasonably tries to use as little RAM as possible.
+For testing, benchmarking, API purposes only!
+</body>
+</html>

Modified: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1441171&r1=1441170&r2=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Thu Jan 31 20:12:57 2013
@@ -36,6 +36,7 @@ import org.apache.lucene.codecs.lucene41
 import org.apache.lucene.codecs.lucene42.Lucene42Codec;
 import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
 import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
+import org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat;
 import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.codecs.memory.DirectPostingsFormat;
 import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
@@ -146,14 +147,15 @@ public class RandomCodec extends Lucene4
         new Lucene42DocValuesFormat(),
         new DiskDocValuesFormat(),
         new SimpleTextDocValuesFormat(),
-        new AssertingDocValuesFormat());
+        new AssertingDocValuesFormat(),
+        new CheapBastardDocValuesFormat());
 
     Collections.shuffle(formats, random);
     Collections.shuffle(dvFormats, random);
 
     // Avoid too many open files:
     formats = formats.subList(0, 4);
-    // only if we get big dvFormats = dvFormats.subList(0, 4);
+    dvFormats = dvFormats.subList(0, 4);
   }
 
   public RandomCodec(Random random) {

Modified: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java?rev=1441171&r1=1441170&r2=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java Thu Jan 31 20:12:57 2013
@@ -31,6 +31,7 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.asserting.AssertingCodec;
+import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
 import org.apache.lucene.codecs.compressing.CompressingCodec;
 import org.apache.lucene.codecs.lucene40.Lucene40Codec;
 import org.apache.lucene.codecs.lucene40.Lucene40RWCodec;
@@ -190,6 +191,8 @@ final class TestRuleSetupAndRestoreClass
       };
     } else if ("SimpleText".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 9 && !shouldAvoidCodec("SimpleText"))) {
       codec = new SimpleTextCodec();
+    } else if ("CheapBastard".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 8 && !shouldAvoidCodec("CheapBastard"))) {
+      codec = new CheapBastardCodec();
     } else if ("Asserting".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 7 && !shouldAvoidCodec("Asserting"))) {
       codec = new AssertingCodec();
     } else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {

Modified: lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec?rev=1441171&r1=1441170&r2=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.Codec Thu Jan 31 20:12:57 2013
@@ -14,6 +14,7 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.asserting.AssertingCodec
+org.apache.lucene.codecs.cheapbastard.CheapBastardCodec
 org.apache.lucene.codecs.compressing.FastCompressingCodec
 org.apache.lucene.codecs.compressing.FastDecompressionCompressingCodec
 org.apache.lucene.codecs.compressing.HighCompressionCompressingCodec

Modified: lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1441171&r1=1441170&r2=1441171&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Thu Jan 31 20:12:57 2013
@@ -14,3 +14,4 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.asserting.AssertingDocValuesFormat
+org.apache.lucene.codecs.cheapbastard.CheapBastardDocValuesFormat
\ No newline at end of file