You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2012/03/07 11:26:45 UTC

svn commit: r1297920 [1/2] - in /lucene/dev/trunk/lucene: core/src/java/org/apache/lucene/codecs/ core/src/java/org/apache/lucene/codecs/lucene40/ core/src/java/org/apache/lucene/codecs/lucene40/values/ core/src/java/org/apache/lucene/codecs/sep/ core/...

Author: simonw
Date: Wed Mar  7 10:26:44 2012
New Revision: 1297920

URL: http://svn.apache.org/viewvc?rev=1297920&view=rev
Log:
LUCENE-3074: simple text doc values

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java
      - copied, changed from r1297903, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesArray.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java
      - copied, changed from r1297903, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesReaderBase.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/BytesRefUtils.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesArray.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesReaderBase.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsProducer.java
Modified:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesWriterBase.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedSortedBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarDerefBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Writer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexableField.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWNormsConsumer.java

Copied: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java (from r1297903, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesArray.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java?p2=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java&p1=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesArray.java&r1=1297903&r2=1297920&rev=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesArray.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesArraySource.java Wed Mar  7 10:26:44 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene40.values;
+package org.apache.lucene.codecs;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -30,13 +30,14 @@ import org.apache.lucene.util.RamUsageEs
 
 /**
  * @lucene.experimental
+ * @lucene.internal
  */
-abstract class DocValuesArray extends Source {
+public abstract class DocValuesArraySource extends Source {
 
-  static final Map<Type, DocValuesArray> TEMPLATES;
+  private static final Map<Type, DocValuesArraySource> TEMPLATES;
 
   static {
-    EnumMap<Type, DocValuesArray> templates = new EnumMap<Type, DocValuesArray>(
+    EnumMap<Type, DocValuesArraySource> templates = new EnumMap<Type, DocValuesArraySource>(
         Type.class);
     templates.put(Type.FIXED_INTS_16, new ShortValues());
     templates.put(Type.FIXED_INTS_32, new IntValues());
@@ -46,37 +47,51 @@ abstract class DocValuesArray extends So
     templates.put(Type.FLOAT_64, new DoubleValues());
     TEMPLATES = Collections.unmodifiableMap(templates);
   }
+  
+  public static DocValuesArraySource forType(Type type) {
+    return TEMPLATES.get(type);
+  }
 
   protected final int bytesPerValue;
 
-  DocValuesArray(int bytesPerValue, Type type) {
+  DocValuesArraySource(int bytesPerValue, Type type) {
     super(type);
     this.bytesPerValue = bytesPerValue;
   }
 
-  public abstract DocValuesArray newFromInput(IndexInput input, int numDocs)
+  @Override
+  public abstract BytesRef getBytes(int docID, BytesRef ref);
+
+  
+  public abstract DocValuesArraySource newFromInput(IndexInput input, int numDocs)
       throws IOException;
+  
+  public abstract DocValuesArraySource newFromArray(Object array);
 
   @Override
   public final boolean hasArray() {
     return true;
   }
 
-  void toBytes(long value, BytesRef bytesRef) {
-    BytesRefUtils.copyLong(bytesRef, value);
+  public void toBytes(long value, BytesRef bytesRef) {
+    copyLong(bytesRef, value);
   }
 
-  void toBytes(double value, BytesRef bytesRef) {
-    BytesRefUtils.copyLong(bytesRef, Double.doubleToRawLongBits(value));
+  public void toBytes(double value, BytesRef bytesRef) {
+    copyLong(bytesRef, Double.doubleToRawLongBits(value));
   }
 
-  final static class ByteValues extends DocValuesArray {
+  final static class ByteValues extends DocValuesArraySource {
     private final byte[] values;
-
+    
     ByteValues() {
       super(1, Type.FIXED_INTS_8);
       values = new byte[0];
     }
+    private ByteValues(byte[] array) {
+      super(1, Type.FIXED_INTS_8);
+      values = array;
+    }
 
     private ByteValues(IndexInput input, int numDocs) throws IOException {
       super(1, Type.FIXED_INTS_8);
@@ -96,24 +111,46 @@ abstract class DocValuesArray extends So
     }
 
     @Override
-    public DocValuesArray newFromInput(IndexInput input, int numDocs)
+    public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
         throws IOException {
       return new ByteValues(input, numDocs);
     }
+    
+    @Override
+    public DocValuesArraySource newFromArray(Object array) {
+      assert array instanceof byte[];
+      return new ByteValues((byte[]) array);
+    }
 
-    void toBytes(long value, BytesRef bytesRef) {
+    public void toBytes(long value, BytesRef bytesRef) {
+      if (bytesRef.bytes.length == 0) {
+        bytesRef.bytes = new byte[1];
+      }
       bytesRef.bytes[0] = (byte) (0xFFL & value);
+      bytesRef.offset = 0;
+      bytesRef.length = 1;
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      toBytes(getInt(docID), ref);
+      return ref;
     }
 
   };
 
-  final static class ShortValues extends DocValuesArray {
+  final static class ShortValues extends DocValuesArraySource {
     private final short[] values;
 
     ShortValues() {
       super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
       values = new short[0];
     }
+    
+    private ShortValues(short[] array) {
+      super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
+      values = array;
+    }
 
     private ShortValues(IndexInput input, int numDocs) throws IOException {
       super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
@@ -135,18 +172,30 @@ abstract class DocValuesArray extends So
     }
 
     @Override
-    public DocValuesArray newFromInput(IndexInput input, int numDocs)
+    public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
         throws IOException {
       return new ShortValues(input, numDocs);
     }
 
-    void toBytes(long value, BytesRef bytesRef) {
-      BytesRefUtils.copyShort(bytesRef, (short) (0xFFFFL & value));
+    public void toBytes(long value, BytesRef bytesRef) {
+      copyShort(bytesRef, (short) (0xFFFFL & value));
+    }
+
+    @Override
+    public DocValuesArraySource newFromArray(Object array) {
+      assert array instanceof short[];
+      return new ShortValues((short[]) array);
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      toBytes(getInt(docID), ref);
+      return ref;
     }
 
   };
 
-  final static class IntValues extends DocValuesArray {
+  final static class IntValues extends DocValuesArraySource {
     private final int[] values;
 
     IntValues() {
@@ -162,6 +211,11 @@ abstract class DocValuesArray extends So
       }
     }
 
+    private IntValues(int[] array) {
+      super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
+      values = array;
+    }
+
     @Override
     public int[] getArray() {
       return values;
@@ -174,18 +228,30 @@ abstract class DocValuesArray extends So
     }
 
     @Override
-    public DocValuesArray newFromInput(IndexInput input, int numDocs)
+    public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
         throws IOException {
       return new IntValues(input, numDocs);
     }
 
-    void toBytes(long value, BytesRef bytesRef) {
-      BytesRefUtils.copyInt(bytesRef, (int) (0xFFFFFFFF & value));
+    public void toBytes(long value, BytesRef bytesRef) {
+      copyInt(bytesRef, (int) (0xFFFFFFFF & value));
+    }
+
+    @Override
+    public DocValuesArraySource newFromArray(Object array) {
+      assert array instanceof int[];
+      return new IntValues((int[]) array);
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      toBytes(getInt(docID), ref);
+      return ref;
     }
 
   };
 
-  final static class LongValues extends DocValuesArray {
+  final static class LongValues extends DocValuesArraySource {
     private final long[] values;
 
     LongValues() {
@@ -201,6 +267,11 @@ abstract class DocValuesArray extends So
       }
     }
 
+    private LongValues(long[] array) {
+      super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
+      values = array;
+    }
+
     @Override
     public long[] getArray() {
       return values;
@@ -213,14 +284,26 @@ abstract class DocValuesArray extends So
     }
 
     @Override
-    public DocValuesArray newFromInput(IndexInput input, int numDocs)
+    public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
         throws IOException {
       return new LongValues(input, numDocs);
     }
 
+    @Override
+    public DocValuesArraySource newFromArray(Object array) {
+      assert array instanceof long[];
+      return new LongValues((long[])array);
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      toBytes(getInt(docID), ref);
+      return ref;
+    }
+
   };
 
-  final static class FloatValues extends DocValuesArray {
+  final static class FloatValues extends DocValuesArraySource {
     private final float[] values;
 
     FloatValues() {
@@ -240,6 +323,11 @@ abstract class DocValuesArray extends So
       }
     }
 
+    private FloatValues(float[] array) {
+      super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
+      values = array;
+    }
+
     @Override
     public float[] getArray() {
       return values;
@@ -252,19 +340,31 @@ abstract class DocValuesArray extends So
     }
     
     @Override
-    void toBytes(double value, BytesRef bytesRef) {
-      BytesRefUtils.copyInt(bytesRef, Float.floatToRawIntBits((float)value));
+    public void toBytes(double value, BytesRef bytesRef) {
+      copyInt(bytesRef, Float.floatToRawIntBits((float)value));
 
     }
 
     @Override
-    public DocValuesArray newFromInput(IndexInput input, int numDocs)
+    public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
         throws IOException {
       return new FloatValues(input, numDocs);
     }
+
+    @Override
+    public DocValuesArraySource newFromArray(Object array) {
+      assert array instanceof float[];
+      return new FloatValues((float[]) array);
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      toBytes(getFloat(docID), ref);
+      return ref;
+    }
   };
 
-  final static class DoubleValues extends DocValuesArray {
+  final static class DoubleValues extends DocValuesArraySource {
     private final double[] values;
 
     DoubleValues() {
@@ -284,6 +384,11 @@ abstract class DocValuesArray extends So
       }
     }
 
+    private DoubleValues(double[] array) {
+      super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
+      values = array;
+    }
+
     @Override
     public double[] getArray() {
       return values;
@@ -296,11 +401,113 @@ abstract class DocValuesArray extends So
     }
 
     @Override
-    public DocValuesArray newFromInput(IndexInput input, int numDocs)
+    public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
         throws IOException {
       return new DoubleValues(input, numDocs);
     }
 
+    @Override
+    public DocValuesArraySource newFromArray(Object array) {
+      assert array instanceof double[];
+      return new DoubleValues((double[]) array);
+    }
+    
+    @Override
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      toBytes(getFloat(docID), ref);
+      return ref;
+    }
+
   };
+  
+  /**
+   * Copies the given long value and encodes it as 8 byte Big-Endian.
+   * <p>
+   * NOTE: this method resets the offset to 0, length to 8 and resizes the
+   * reference array if needed.
+   */
+  public static void copyLong(BytesRef ref, long value) {
+    if (ref.bytes.length < 8) {
+      ref.bytes = new byte[8];
+    }
+    copyInternal(ref, (int) (value >> 32), ref.offset = 0);
+    copyInternal(ref, (int) value, 4);
+    ref.length = 8;
+  }
+
+  /**
+   * Copies the given int value and encodes it as 4 byte Big-Endian.
+   * <p>
+   * NOTE: this method resets the offset to 0, length to 4 and resizes the
+   * reference array if needed.
+   */
+  public static void copyInt(BytesRef ref, int value) {
+    if (ref.bytes.length < 4) {
+      ref.bytes = new byte[4];
+    }
+    copyInternal(ref, value, ref.offset = 0);
+    ref.length = 4;
+    
+  }
+
+  /**
+   * Copies the given short value and encodes it as a 2 byte Big-Endian.
+   * <p>
+   * NOTE: this method resets the offset to 0, length to 2 and resizes the
+   * reference array if needed.
+   */
+  public static void copyShort(BytesRef ref, short value) {
+    if (ref.bytes.length < 2) {
+      ref.bytes = new byte[2];
+    }
+    ref.offset = 0;
+    ref.bytes[ref.offset] = (byte) (value >> 8);
+    ref.bytes[ref.offset + 1] = (byte) (value);
+    ref.length = 2;
+  }
+
+  private static void copyInternal(BytesRef ref, int value, int startOffset) {
+    ref.bytes[startOffset] = (byte) (value >> 24);
+    ref.bytes[startOffset + 1] = (byte) (value >> 16);
+    ref.bytes[startOffset + 2] = (byte) (value >> 8);
+    ref.bytes[startOffset + 3] = (byte) (value);
+  }
+
+  /**
+   * Converts 2 consecutive bytes from the current offset to a short. Bytes are
+   * interpreted as Big-Endian (most significant bit first)
+   * <p>
+   * NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
+   */
+  public static short asShort(BytesRef b) {
+    return (short) (0xFFFF & ((b.bytes[b.offset] & 0xFF) << 8) | (b.bytes[b.offset + 1] & 0xFF));
+  }
+
+  /**
+   * Converts 4 consecutive bytes from the current offset to an int. Bytes are
+   * interpreted as Big-Endian (most significant bit first)
+   * <p>
+   * NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
+   */
+  public static int asInt(BytesRef b) {
+    return asIntInternal(b, b.offset);
+  }
+
+  /**
+   * Converts 8 consecutive bytes from the current offset to a long. Bytes are
+   * interpreted as Big-Endian (most significant bit first)
+   * <p>
+   * NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
+   */
+  public static long asLong(BytesRef b) {
+    return (((long) asIntInternal(b, b.offset) << 32) | asIntInternal(b,
+        b.offset + 4) & 0xFFFFFFFFL);
+  }
+
+  private static int asIntInternal(BytesRef b, int pos) {
+    return ((b.bytes[pos++] & 0xFF) << 24) | ((b.bytes[pos++] & 0xFF) << 16)
+        | ((b.bytes[pos++] & 0xFF) << 8) | (b.bytes[pos] & 0xFF);
+  }
+
 
-}
+}
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Wed Mar  7 10:26:44 2012
@@ -22,6 +22,7 @@ import org.apache.lucene.document.DocVal
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.util.Bits;
@@ -40,6 +41,7 @@ public abstract class DocValuesConsumer 
 
   protected final BytesRef spare = new BytesRef();
 
+  protected abstract Type getType();
   /**
    * Adds the given {@link IndexableField} instance to this
    * {@link DocValuesConsumer}
@@ -110,7 +112,7 @@ public abstract class DocValuesConsumer 
     final Source source = reader.getDirectSource();
     assert source != null;
     int docID = docBase;
-    final DocValues.Type type = reader.type();
+    final Type type = getType();
     final Field scratchField;
     switch(type) {
     case VAR_INTS:
@@ -160,7 +162,7 @@ public abstract class DocValuesConsumer 
    */
   protected void mergeDoc(Field scratchField, Source source, int docID, int sourceDoc)
       throws IOException {
-    switch(source.type()) {
+    switch(getType()) {
     case BYTES_FIXED_DEREF:
     case BYTES_FIXED_SORTED:
     case BYTES_FIXED_STRAIGHT:

Copied: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java (from r1297903, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesReaderBase.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java?p2=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java&p1=lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesReaderBase.java&r1=1297903&r2=1297920&rev=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesReaderBase.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java Wed Mar  7 10:26:44 2012
@@ -1,4 +1,4 @@
-package org.apache.lucene.codecs.lucene40.values;
+package org.apache.lucene.codecs;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -24,10 +24,6 @@ import java.util.Comparator;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.lucene.codecs.PerDocProducer;
-import org.apache.lucene.codecs.lucene40.values.Bytes;
-import org.apache.lucene.codecs.lucene40.values.Floats;
-import org.apache.lucene.codecs.lucene40.values.Ints;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.DocValues;
@@ -40,7 +36,7 @@ import org.apache.lucene.util.BytesRef;
  * Abstract base class for PerDocProducer implementations
  * @lucene.experimental
  */
-public abstract class DocValuesReaderBase extends PerDocProducer {
+public abstract class PerDocProducerBase extends PerDocProducer {
 
   protected abstract void closeInternal(Collection<? extends Closeable> closeables) throws IOException;
   protected abstract Map<String, DocValues> docValues();
@@ -70,9 +66,7 @@ public abstract class DocValuesReaderBas
       for (FieldInfo fieldInfo : fieldInfos) {
         if (canLoad(fieldInfo)) {
           final String field = fieldInfo.name;
-          // TODO can we have a compound file per segment and codec for
-          // docvalues?
-          final String id = DocValuesWriterBase.docValuesId(segment,
+          final String id = docValuesId(segment,
               fieldInfo.number);
           values.put(field,
               loadDocValues(docCount, dir, id, getDocValuesType(fieldInfo), context));
@@ -100,6 +94,10 @@ public abstract class DocValuesReaderBas
     return infos.anyDocValuesFields();
   }
   
+  public static String docValuesId(String segmentsName, int fieldId) {
+    return segmentsName + "_" + fieldId;
+  }
+  
   /**
    * Loads a {@link DocValues} instance depending on the given {@link Type}.
    * Codecs that use different implementations for a certain {@link Type} can
@@ -119,33 +117,6 @@ public abstract class DocValuesReaderBas
    * @throws IllegalArgumentException
    *           if the given {@link Type} is not supported
    */
-  protected DocValues loadDocValues(int docCount, Directory dir, String id,
-      DocValues.Type type, IOContext context) throws IOException {
-    switch (type) {
-    case FIXED_INTS_16:
-    case FIXED_INTS_32:
-    case FIXED_INTS_64:
-    case FIXED_INTS_8:
-    case VAR_INTS:
-      return Ints.getValues(dir, id, docCount, type, context);
-    case FLOAT_32:
-      return Floats.getValues(dir, id, docCount, context, type);
-    case FLOAT_64:
-      return Floats.getValues(dir, id, docCount, context, type);
-    case BYTES_FIXED_STRAIGHT:
-      return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
-    case BYTES_FIXED_DEREF:
-      return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
-    case BYTES_FIXED_SORTED:
-      return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
-    case BYTES_VAR_STRAIGHT:
-      return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
-    case BYTES_VAR_DEREF:
-      return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
-    case BYTES_VAR_SORTED:
-      return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
-    default:
-      throw new IllegalStateException("unrecognized index values mode " + type);
-    }
-  }
+  protected abstract DocValues loadDocValues(int docCount, Directory dir, String id,
+      DocValues.Type type, IOContext context) throws IOException;
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesProducer.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesProducer.java Wed Mar  7 10:26:44 2012
@@ -24,19 +24,24 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.lucene.codecs.lucene40.values.DocValuesReaderBase;
+import org.apache.lucene.codecs.PerDocProducerBase;
+import org.apache.lucene.codecs.lucene40.values.Bytes;
+import org.apache.lucene.codecs.lucene40.values.Floats;
+import org.apache.lucene.codecs.lucene40.values.Ints;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.IOUtils;
 
 /**
  * Default PerDocProducer implementation that uses compound file.
  * @lucene.experimental
  */
-public class Lucene40DocValuesProducer extends DocValuesReaderBase {
+public class Lucene40DocValuesProducer extends PerDocProducerBase {
   protected final TreeMap<String,DocValues> docValues;
   private final Directory cfs;
   /**
@@ -71,4 +76,35 @@ public class Lucene40DocValuesProducer e
       IOUtils.close(closeables);
     }
   }
+
+  @Override
+  protected DocValues loadDocValues(int docCount, Directory dir, String id,
+      Type type, IOContext context) throws IOException {
+      switch (type) {
+      case FIXED_INTS_16:
+      case FIXED_INTS_32:
+      case FIXED_INTS_64:
+      case FIXED_INTS_8:
+      case VAR_INTS:
+        return Ints.getValues(dir, id, docCount, type, context);
+      case FLOAT_32:
+        return Floats.getValues(dir, id, docCount, context, type);
+      case FLOAT_64:
+        return Floats.getValues(dir, id, docCount, context, type);
+      case BYTES_FIXED_STRAIGHT:
+        return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
+      case BYTES_FIXED_DEREF:
+        return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
+      case BYTES_FIXED_SORTED:
+        return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
+      case BYTES_VAR_STRAIGHT:
+        return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
+      case BYTES_VAR_DEREF:
+        return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
+      case BYTES_VAR_SORTED:
+        return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
+      default:
+        throw new IllegalStateException("unrecognized index values mode " + type);
+      }
+    }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java Wed Mar  7 10:26:44 2012
@@ -23,7 +23,6 @@ import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DocValues.SortedSource;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
@@ -64,7 +63,7 @@ import org.apache.lucene.util.packed.Pac
  * 
  * @lucene.experimental
  */
-final class Bytes {
+public final class Bytes {
 
   static final String DV_SEGMENT_SUFFIX = "dv";
 
@@ -242,8 +241,8 @@ final class Bytes {
     private final IOContext context;
 
     protected BytesWriterBase(Directory dir, String id, String codecName,
-        int version, Counter bytesUsed, IOContext context) throws IOException {
-      super(bytesUsed);
+        int version, Counter bytesUsed, IOContext context, Type type) throws IOException {
+      super(bytesUsed, type);
       this.id = id;
       this.dir = dir;
       this.codecName = codecName;
@@ -292,25 +291,11 @@ final class Bytes {
       }
       return idxOut;
     }
-    /**
-     * Must be called only with increasing docIDs. It's OK for some docIDs to be
-     * skipped; they will be filled with 0 bytes.
-     */
-    protected
-    abstract void add(int docID, BytesRef bytes) throws IOException;
 
-    @Override
-    public abstract void finish(int docCount) throws IOException;
 
     @Override
-    protected void mergeDoc(Field scratchField, Source source, int docID, int sourceDoc) throws IOException {
-      add(docID, source.getBytes(sourceDoc, bytesRef));
-    }
+    public abstract void finish(int docCount) throws IOException;
 
-    @Override
-    public void add(int docID, IndexableField docValue) throws IOException {
-      add(docID, docValue.binaryValue());
-    }
   }
 
   /**
@@ -393,22 +378,22 @@ final class Bytes {
     protected long maxBytes = 0;
     
     protected DerefBytesWriterBase(Directory dir, String id, String codecName,
-        int codecVersion, Counter bytesUsed, IOContext context)
+        int codecVersion, Counter bytesUsed, IOContext context, Type type)
         throws IOException {
       this(dir, id, codecName, codecVersion, new DirectTrackingAllocator(
-          ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, false);
+          ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, false, type);
     }
 
     protected DerefBytesWriterBase(Directory dir, String id, String codecName,
-                                   int codecVersion, Counter bytesUsed, IOContext context, boolean fasterButMoreRam)
+                                   int codecVersion, Counter bytesUsed, IOContext context, boolean fasterButMoreRam, Type type)
         throws IOException {
       this(dir, id, codecName, codecVersion, new DirectTrackingAllocator(
-          ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, fasterButMoreRam);
+          ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, fasterButMoreRam,type);
     }
 
     protected DerefBytesWriterBase(Directory dir, String id, String codecName, int codecVersion, Allocator allocator,
-        Counter bytesUsed, IOContext context, boolean fasterButMoreRam) throws IOException {
-      super(dir, id, codecName, codecVersion, bytesUsed, context);
+        Counter bytesUsed, IOContext context, boolean fasterButMoreRam, Type type) throws IOException {
+      super(dir, id, codecName, codecVersion, bytesUsed, context, type);
       hash = new BytesRefHash(new ByteBlockPool(allocator),
           BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
               BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
@@ -430,7 +415,9 @@ final class Bytes {
     }
 
     @Override
-    protected void add(int docID, BytesRef bytes) throws IOException {
+    public void add(int docID, IndexableField value) throws IOException {
+      BytesRef bytes = value.binaryValue();
+      assert bytes != null;
       if (bytes.length == 0) { // default value - skip it
         return;
       }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesWriterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesWriterBase.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesWriterBase.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/DocValuesWriterBase.java Wed Mar  7 10:26:44 2012
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Comparator;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.PerDocProducerBase;
 import org.apache.lucene.codecs.PerDocConsumer;
 import org.apache.lucene.codecs.lucene40.values.Writer;
 import org.apache.lucene.index.FieldInfo;
@@ -81,14 +82,10 @@ public abstract class DocValuesWriterBas
   @Override
   public DocValuesConsumer addValuesField(Type valueType, FieldInfo field) throws IOException {
     return Writer.create(valueType,
-        docValuesId(segmentName, field.number), 
+        PerDocProducerBase.docValuesId(segmentName, field.number), 
         getDirectory(), getComparator(), bytesUsed, context, fasterButMoreRam);
   }
 
-  public static String docValuesId(String segmentsName, int fieldId) {
-    return segmentsName + "_" + fieldId;
-  }
-  
   
   public Comparator<BytesRef> getComparator() throws IOException {
     return BytesRef.getUTF8SortedAsUnicodeComparator();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedDerefBytesImpl.java Wed Mar  7 10:26:44 2012
@@ -46,7 +46,7 @@ class FixedDerefBytesImpl {
   public static class Writer extends DerefBytesWriterBase {
     public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.BYTES_FIXED_DEREF);
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedSortedBytesImpl.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedSortedBytesImpl.java Wed Mar  7 10:26:44 2012
@@ -58,7 +58,7 @@ class FixedSortedBytesImpl {
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         Counter bytesUsed, IOContext context, boolean fasterButMoreRam) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, fasterButMoreRam);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, fasterButMoreRam, Type.BYTES_FIXED_SORTED);
       this.comp = comp;
     }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java Wed Mar  7 10:26:44 2012
@@ -22,10 +22,12 @@ import java.io.IOException;
 import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
 import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
 import org.apache.lucene.codecs.lucene40.values.Bytes.BytesWriterBase;
+import org.apache.lucene.document.DocValuesField;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -52,6 +54,7 @@ class FixedStraightBytesImpl {
   static final int VERSION_CURRENT = VERSION_START;
   
   static abstract class FixedBytesWriterBase extends BytesWriterBase {
+    protected final DocValuesField bytesSpareField = new DocValuesField("", new BytesRef(), Type.BYTES_FIXED_STRAIGHT);
     protected int lastDocID = -1;
     // start at -1 if the first added value is > 0
     protected int size = -1;
@@ -60,13 +63,20 @@ class FixedStraightBytesImpl {
 
     protected FixedBytesWriterBase(Directory dir, String id, String codecName,
         int version, Counter bytesUsed, IOContext context) throws IOException {
-      super(dir, id, codecName, version, bytesUsed, context);
+     this(dir, id, codecName, version, bytesUsed, context, Type.BYTES_FIXED_STRAIGHT);
+    }
+    
+    protected FixedBytesWriterBase(Directory dir, String id, String codecName,
+        int version, Counter bytesUsed, IOContext context, Type type) throws IOException {
+      super(dir, id, codecName, version, bytesUsed, context, type);
       pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
       pool.nextBuffer();
     }
     
     @Override
-    protected void add(int docID, BytesRef bytes) throws IOException {
+    public void add(int docID, IndexableField value) throws IOException {
+      final BytesRef bytes = value.binaryValue();
+      assert bytes != null;
       assert lastDocID < docID;
 
       if (size == -1) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java Wed Mar  7 10:26:44 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene4
  */
 import java.io.IOException;
 
+import org.apache.lucene.codecs.DocValuesArraySource;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
@@ -39,7 +40,7 @@ import org.apache.lucene.util.IOUtils;
  * 
  * @lucene.experimental
  */
-class Floats {
+public class Floats {
   
   protected static final String CODEC_NAME = "Floats";
   protected static final int VERSION_START = 0;
@@ -69,27 +70,17 @@ class Floats {
   final static class FloatsWriter extends FixedStraightBytesImpl.Writer {
    
     private final int size; 
-    private final DocValuesArray template;
+    private final DocValuesArraySource template;
     public FloatsWriter(Directory dir, String id, Counter bytesUsed,
         IOContext context, Type type) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
       size = typeToSize(type);
       this.bytesRef = new BytesRef(size);
       bytesRef.length = size;
-      template = DocValuesArray.TEMPLATES.get(type);
+      template = DocValuesArraySource.forType(type);
       assert template != null;
     }
     
-    protected void add(int docID, double v) throws IOException {
-      template.toBytes(v, bytesRef);
-      add(docID, bytesRef);
-    }
-    
-    @Override
-    public void add(int docID, IndexableField docValue) throws IOException {
-      add(docID, docValue.numericValue().doubleValue());
-    }
-    
     @Override
     protected boolean tryBulkMerge(DocValues docValues) {
       // only bulk merge if value type is the same otherwise size differs
@@ -97,6 +88,13 @@ class Floats {
     }
     
     @Override
+    public void add(int docID, IndexableField value) throws IOException {
+      template.toBytes(value.numericValue().doubleValue(), bytesRef);
+      bytesSpareField.setBytesValue(bytesRef);
+      super.add(docID, bytesSpareField);
+    }
+    
+    @Override
     protected void setMergeBytes(Source source, int sourceDoc) {
       final double value = source.getFloat(sourceDoc);
       template.toBytes(value, bytesRef);
@@ -104,11 +102,11 @@ class Floats {
   }
   
   final static class FloatsReader extends FixedStraightBytesImpl.FixedStraightReader {
-    final DocValuesArray arrayTemplate;
+    final DocValuesArraySource arrayTemplate;
     FloatsReader(Directory dir, String id, int maxDoc, IOContext context, Type type)
         throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, type);
-      arrayTemplate = DocValuesArray.TEMPLATES.get(type);
+      arrayTemplate = DocValuesArraySource.forType(type);
       assert size == 4 || size == 8: "wrong size=" + size + " type=" + type + " id=" + id;
     }
     

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java Wed Mar  7 10:26:44 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene4
 
 import java.io.IOException;
 
+import org.apache.lucene.codecs.DocValuesArraySource;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
@@ -36,7 +37,7 @@ import org.apache.lucene.util.IOUtils;
  * 
  * @lucene.experimental
  */
-final class Ints {
+public final class Ints {
   protected static final String CODEC_NAME = "Ints";
   protected static final int VERSION_START = 0;
   protected static final int VERSION_CURRENT = VERSION_START;
@@ -88,7 +89,7 @@ final class Ints {
 
 
   static class IntsWriter extends FixedStraightBytesImpl.Writer {
-    private final DocValuesArray template;
+    private final DocValuesArraySource template;
 
     public IntsWriter(Directory dir, String id, Counter bytesUsed,
         IOContext context, Type valueType) throws IOException {
@@ -101,17 +102,7 @@ final class Ints {
       size = typeToSize(valueType);
       this.bytesRef = new BytesRef(size);
       bytesRef.length = size;
-      template = DocValuesArray.TEMPLATES.get(valueType);
-    }
-    
-    protected void add(int docID, long v) throws IOException {
-      template.toBytes(v, bytesRef);
-      add(docID, bytesRef);
-    }
-
-    @Override
-    public void add(int docID, IndexableField docValue) throws IOException {
-      add(docID, docValue.numericValue().longValue());
+      template = DocValuesArraySource.forType(valueType);
     }
     
     @Override
@@ -121,6 +112,13 @@ final class Ints {
     }
     
     @Override
+    public void add(int docID, IndexableField value) throws IOException {
+      template.toBytes(value.numericValue().longValue(), bytesRef);
+      bytesSpareField.setBytesValue(bytesRef);
+      super.add(docID, bytesSpareField);
+    }
+
+    @Override
     protected boolean tryBulkMerge(DocValues docValues) {
       // only bulk merge if value type is the same otherwise size differs
       return super.tryBulkMerge(docValues) && docValues.type() == template.type();
@@ -128,13 +126,13 @@ final class Ints {
   }
   
   final static class IntsReader extends FixedStraightBytesImpl.FixedStraightReader {
-    private final DocValuesArray arrayTemplate;
+    private final DocValuesArraySource arrayTemplate;
 
     IntsReader(Directory dir, String id, int maxDoc, IOContext context, Type type)
         throws IOException {
       super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc,
           context, type);
-      arrayTemplate = DocValuesArray.TEMPLATES.get(type);
+      arrayTemplate = DocValuesArraySource.forType(type);
       assert arrayTemplate != null;
       assert type == sizeToType(size);
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java Wed Mar  7 10:26:44 2012
@@ -18,9 +18,8 @@ package org.apache.lucene.codecs.lucene4
  */
 import java.io.IOException;
 
-import org.apache.lucene.codecs.lucene40.values.DocValuesArray.LongValues;
+import org.apache.lucene.codecs.DocValuesArraySource;
 import org.apache.lucene.codecs.lucene40.values.FixedStraightBytesImpl.FixedBytesWriterBase;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocValues;
@@ -59,27 +58,10 @@ class PackedIntValues {
 
     protected PackedIntsWriter(Directory dir, String id, Counter bytesUsed,
         IOContext context) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.VAR_INTS);
       bytesRef = new BytesRef(8);
     }
-
-    protected void add(int docID, long v) throws IOException {
-      assert lastDocId < docID;
-      if (!started) {
-        started = true;
-        minValue = maxValue = v;
-      } else {
-        if (v < minValue) {
-          minValue = v;
-        } else if (v > maxValue) {
-          maxValue = v;
-        }
-      }
-      lastDocId = docID;
-      BytesRefUtils.copyLong(bytesRef, v);
-      add(docID, bytesRef);
-    }
-
+    
     @Override
     public void finish(int docCount) throws IOException {
       boolean success = false;
@@ -112,13 +94,6 @@ class PackedIntValues {
       }
     }
 
-    @Override
-    protected void mergeDoc(Field scratchField, Source source, int docID, int sourceDoc) throws IOException {
-      assert docID > lastDocId : "docID: " + docID
-          + " must be greater than the last added doc id: " + lastDocId;
-        add(docID, source.getInt(sourceDoc));
-    }
-
     private void writePackedInts(IndexOutput datOut, int docCount) throws IOException {
       datOut.writeLong(minValue);
       
@@ -149,10 +124,25 @@ class PackedIntValues {
       }
       w.finish();
     }
-
+    
     @Override
     public void add(int docID, IndexableField docValue) throws IOException {
-      add(docID, docValue.numericValue().longValue());
+      final long v = docValue.numericValue().longValue();
+      assert lastDocId < docID;
+      if (!started) {
+        started = true;
+        minValue = maxValue = v;
+      } else {
+        if (v < minValue) {
+          minValue = v;
+        } else if (v > maxValue) {
+          maxValue = v;
+        }
+      }
+      lastDocId = docID;
+      DocValuesArraySource.copyLong(bytesRef, v);
+      bytesSpareField.setBytesValue(bytesRef);
+      super.add(docID, bytesSpareField);
     }
   }
 
@@ -164,7 +154,7 @@ class PackedIntValues {
     private final IndexInput datIn;
     private final byte type;
     private final int numDocs;
-    private final LongValues values;
+    private final DocValuesArraySource values;
 
     protected PackedIntsReader(Directory dir, String id, int numDocs,
         IOContext context) throws IOException {
@@ -176,7 +166,7 @@ class PackedIntValues {
       try {
         CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
         type = datIn.readByte();
-        values = type == FIXED_64 ? new LongValues() : null;
+        values = type == FIXED_64 ?  DocValuesArraySource.forType(Type.FIXED_INTS_64) : null;
         success = true;
       } finally {
         if (!success) {
@@ -247,7 +237,7 @@ class PackedIntValues {
     @Override
     public BytesRef getBytes(int docID, BytesRef ref) {
       ref.grow(8);
-      BytesRefUtils.copyLong(ref, getInt(docID));
+      DocValuesArraySource.copyLong(ref, getInt(docID));
       return ref;
     }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarDerefBytesImpl.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarDerefBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarDerefBytesImpl.java Wed Mar  7 10:26:44 2012
@@ -57,7 +57,7 @@ class VarDerefBytesImpl {
   static class Writer extends DerefBytesWriterBase {
     public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.BYTES_VAR_DEREF);
       size = 0;
     }
     

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java Wed Mar  7 10:26:44 2012
@@ -59,7 +59,7 @@ final class VarSortedBytesImpl {
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         Counter bytesUsed, IOContext context, boolean fasterButMoreRam) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, fasterButMoreRam);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, fasterButMoreRam, Type.BYTES_VAR_SORTED);
       this.comp = comp;
       size = 0;
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java Wed Mar  7 10:26:44 2012
@@ -26,6 +26,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -63,7 +64,7 @@ class VarStraightBytesImpl {
     private boolean merge = false;
     public Writer(Directory dir, String id, Counter bytesUsed, IOContext context)
         throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.BYTES_VAR_STRAIGHT);
       pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
       docToAddress = new long[1];
       pool.nextBuffer(); // init
@@ -84,7 +85,9 @@ class VarStraightBytesImpl {
     }
 
     @Override
-    protected void add(int docID, BytesRef bytes) throws IOException {
+    public void add(int docID, IndexableField value) throws IOException {
+      final BytesRef bytes = value.binaryValue();
+      assert bytes != null;
       assert !merge;
       if (bytes.length == 0) {
         return; // default

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Writer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Writer.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Writer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Writer.java Wed Mar  7 10:26:44 2012
@@ -40,6 +40,7 @@ import org.apache.lucene.util.Counter;
  */
 abstract class Writer extends DocValuesConsumer {
   protected final Counter bytesUsed;
+  protected Type type;
 
   /**
    * Creates a new {@link Writer}.
@@ -49,9 +50,19 @@ abstract class Writer extends DocValuesC
    *          internally allocated memory. All tracked bytes must be released
    *          once {@link #finish(int)} has been called.
    */
-  protected Writer(Counter bytesUsed) {
+  protected Writer(Counter bytesUsed, Type type) {
     this.bytesUsed = bytesUsed;
+    this.type = type;
   }
+  
+  
+
+  @Override
+  protected Type getType() {
+    return type;
+  }
+
+
 
   /**
    * Factory method to create a {@link Writer} instance for a given type. This

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java Wed Mar  7 10:26:44 2012
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.lucene.codecs.PerDocProducerBase;
 import org.apache.lucene.codecs.lucene40.values.DocValuesWriterBase;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
@@ -58,7 +59,7 @@ public class SepDocValuesConsumer extend
   private static void files(Directory dir,FieldInfos fieldInfos, String segmentName, Set<String> files)  {
     for (FieldInfo fieldInfo : fieldInfos) {
       if (fieldInfo.hasDocValues()) {
-        String filename = docValuesId(segmentName, fieldInfo.number);
+        String filename = PerDocProducerBase.docValuesId(segmentName, fieldInfo.number);
         switch (fieldInfo.getDocValuesType()) {
           case BYTES_FIXED_DEREF:
           case BYTES_VAR_DEREF:

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesProducer.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesProducer.java Wed Mar  7 10:26:44 2012
@@ -22,16 +22,22 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.lucene.codecs.lucene40.values.DocValuesReaderBase;
+import org.apache.lucene.codecs.PerDocProducerBase;
+import org.apache.lucene.codecs.lucene40.values.Bytes;
+import org.apache.lucene.codecs.lucene40.values.Floats;
+import org.apache.lucene.codecs.lucene40.values.Ints;
 import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.IOUtils;
 
 /**
  * Implementation of PerDocProducer that uses separate files.
  * @lucene.experimental
  */
-public class SepDocValuesProducer extends DocValuesReaderBase {
+public class SepDocValuesProducer extends PerDocProducerBase {
   private final TreeMap<String, DocValues> docValues;
 
   /**
@@ -51,4 +57,35 @@ public class SepDocValuesProducer extend
   protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
     IOUtils.close(closeables);
   }
+
+  @Override
+  protected DocValues loadDocValues(int docCount, Directory dir, String id,
+      Type type, IOContext context) throws IOException {
+      switch (type) {
+      case FIXED_INTS_16:
+      case FIXED_INTS_32:
+      case FIXED_INTS_64:
+      case FIXED_INTS_8:
+      case VAR_INTS:
+        return Ints.getValues(dir, id, docCount, type, context);
+      case FLOAT_32:
+        return Floats.getValues(dir, id, docCount, context, type);
+      case FLOAT_64:
+        return Floats.getValues(dir, id, docCount, context, type);
+      case BYTES_FIXED_STRAIGHT:
+        return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
+      case BYTES_FIXED_DEREF:
+        return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
+      case BYTES_FIXED_SORTED:
+        return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
+      case BYTES_VAR_STRAIGHT:
+        return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
+      case BYTES_VAR_DEREF:
+        return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
+      case BYTES_VAR_SORTED:
+        return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
+      default:
+        throw new IllegalStateException("unrecognized index values mode " + type);
+      }
+    }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java Wed Mar  7 10:26:44 2012
@@ -26,7 +26,6 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.SegmentInfosFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
 
 /**
  * plain text index format.
@@ -41,7 +40,7 @@ public final class SimpleTextCodec exten
   private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat();
   private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
   // TODO: need a plain-text impl
-  private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
+  private final DocValuesFormat docValues = new SimpleTextDocValuesFormat();
   // TODO: need a plain-text impl (using the above)
   private final NormsFormat normsFormat = new SimpleTextNormsFormat();
   private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java?rev=1297920&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java Wed Mar  7 10:26:44 2012
@@ -0,0 +1,288 @@
+package org.apache.lucene.codecs.simpletext;
+/**
+ * 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.DocValuesArraySource;
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * @lucene.experimental
+ */
+public class SimpleTextDocValuesConsumer extends DocValuesConsumer {
+
+  static final BytesRef ZERO_DOUBLE = new BytesRef(Double.toString(0d));
+  static final BytesRef ZERO_INT = new BytesRef(Integer.toString(0));
+  static final BytesRef HEADER = new BytesRef("SimpleTextDocValues"); 
+
+  static final BytesRef END = new BytesRef("END");
+  static final BytesRef VALUE_SIZE = new BytesRef("valuesize ");
+  static final BytesRef DOC = new BytesRef("  doc ");
+  static final BytesRef VALUE = new BytesRef("    value ");
+  protected BytesRef scratch = new BytesRef();
+  protected int maxDocId = -1;
+  protected final String segment;
+  protected final Directory dir;
+  protected final IOContext ctx;
+  protected final Type type;
+  protected final BytesRefHash hash;
+  private int[] ords;
+  private int fixedSize = Integer.MIN_VALUE;
+  private BytesRef zeroBytes;
+  private final String segmentSuffix;
+  
+
+  public SimpleTextDocValuesConsumer(String segment, Directory dir,
+      IOContext ctx, Type type, String segmentSuffix) {
+    this.ctx = ctx;
+    this.dir = dir;
+    this.segment = segment;
+    this.type = type;
+    hash = new BytesRefHash();
+    ords = new int[0];
+    this.segmentSuffix = segmentSuffix;
+
+  }
+
+  @Override
+  public void add(int docID, IndexableField value) throws IOException {
+    assert docID >= 0;
+    int ord = -1;
+    int vSize = -1;
+    switch (type) {
+    case BYTES_FIXED_DEREF:
+    case BYTES_FIXED_SORTED:
+    case BYTES_FIXED_STRAIGHT:
+      vSize = value.binaryValue().length;
+      ord = hash.add(value.binaryValue());
+      break;
+    case BYTES_VAR_DEREF:
+    case BYTES_VAR_SORTED:
+    case BYTES_VAR_STRAIGHT:
+      vSize = -1;
+      try {
+      ord = hash.add(value.binaryValue());
+      } catch (NullPointerException e) {
+        System.err.println();
+      }
+      break;
+    case FIXED_INTS_16:
+      vSize = 2;
+      scratch.grow(2);
+      DocValuesArraySource.copyShort(scratch, value.numericValue().shortValue());
+      ord = hash.add(scratch);
+      break;
+    case FIXED_INTS_32:
+      vSize = 4;
+      scratch.grow(4);
+      DocValuesArraySource.copyInt(scratch, value.numericValue().intValue());
+      ord = hash.add(scratch);
+      break;
+    case FIXED_INTS_8:
+      vSize = 1;
+      scratch.grow(1); 
+      scratch.bytes[scratch.offset] = value.numericValue().byteValue();
+      scratch.length = 1;
+      ord = hash.add(scratch);
+      break;
+    case FIXED_INTS_64:
+      vSize = 8;
+    case VAR_INTS:
+      scratch.grow(8);
+      DocValuesArraySource.copyLong(scratch, value.numericValue().longValue());
+      ord = hash.add(scratch);
+      break;
+    case FLOAT_32:
+      vSize = 4;
+      scratch.grow(4);
+      DocValuesArraySource.copyInt(scratch,
+          Float.floatToRawIntBits(value.numericValue().floatValue()));
+      ord = hash.add(scratch);
+      break;
+    case FLOAT_64:
+      vSize = 8;
+      scratch.grow(8);
+      DocValuesArraySource.copyLong(scratch,
+          Double.doubleToRawLongBits(value.numericValue().doubleValue()));
+      ord = hash.add(scratch);
+      break;
+
+    }
+    
+    if (fixedSize == Integer.MIN_VALUE) {
+      assert maxDocId == -1;
+      fixedSize = vSize;
+    } else {
+      if (fixedSize != vSize) {
+        throw new IllegalArgumentException("value size must be " + fixedSize + " but was: " + vSize);
+      }
+    }
+    maxDocId = Math.max(docID, maxDocId);
+    ords = grow(ords, docID);
+    
+    ords[docID] = (ord < 0 ? (-ord)-1 : ord) + 1;
+  }
+  
+  protected BytesRef getHeader() {
+    return HEADER;
+  }
+
+  private int[] grow(int[] array, int upto) {
+    if (array.length <= upto) {
+      return ArrayUtil.grow(array, 1 + upto);
+    }
+    return array;
+  }
+
+  private void prepareFlush(int docCount) {
+    assert ords != null;
+    ords = grow(ords, docCount);
+  }
+
+  @Override
+  public void finish(int docCount) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segment, "",
+        segmentSuffix);
+    IndexOutput output = dir.createOutput(fileName, ctx);
+    boolean success = false;
+    BytesRef spare = new BytesRef();
+    try {
+      SimpleTextUtil.write(output, getHeader());
+      SimpleTextUtil.writeNewline(output);
+      SimpleTextUtil.write(output, VALUE_SIZE);
+      SimpleTextUtil.write(output, Integer.toString(this.fixedSize), scratch);
+      SimpleTextUtil.writeNewline(output);
+      prepareFlush(docCount);
+      for (int i = 0; i < docCount; i++) {
+        SimpleTextUtil.write(output, DOC);
+        SimpleTextUtil.write(output, Integer.toString(i), scratch);
+        SimpleTextUtil.writeNewline(output);
+        SimpleTextUtil.write(output, VALUE);
+        writeDoc(output, i, spare);
+        SimpleTextUtil.writeNewline(output);
+      }
+      SimpleTextUtil.write(output, END);
+      SimpleTextUtil.writeNewline(output);
+      success = true;
+    } finally {
+      hash.close();
+      if (success) {
+        IOUtils.close(output);
+      } else {
+        IOUtils.closeWhileHandlingException(output);
+      }
+    }
+  }
+
+  protected void writeDoc(IndexOutput output, int docId, BytesRef spare) throws IOException {
+    int ord = ords[docId] - 1;
+    if (ord != -1) {
+      assert ord >= 0;
+      hash.get(ord, spare);
+
+      switch (type) {
+      case BYTES_FIXED_DEREF:
+      case BYTES_FIXED_SORTED:
+      case BYTES_FIXED_STRAIGHT:
+      case BYTES_VAR_DEREF:
+      case BYTES_VAR_SORTED:
+      case BYTES_VAR_STRAIGHT:
+        SimpleTextUtil.write(output, spare);
+        break;
+      case FIXED_INTS_16:
+        SimpleTextUtil.write(output,
+            Short.toString(DocValuesArraySource.asShort(spare)), scratch);
+        break;
+      case FIXED_INTS_32:
+        SimpleTextUtil.write(output,
+            Integer.toString(DocValuesArraySource.asInt(spare)), scratch);
+        break;
+      case VAR_INTS:
+      case FIXED_INTS_64:
+        SimpleTextUtil.write(output,
+            Long.toString(DocValuesArraySource.asLong(spare)), scratch);
+        break;
+      case FIXED_INTS_8:
+        assert spare.length == 1 : spare.length;
+        SimpleTextUtil.write(output,
+            Integer.toString(spare.bytes[spare.offset]), scratch);
+        break;
+      case FLOAT_32:
+        float valueFloat = Float.intBitsToFloat(DocValuesArraySource.asInt(spare));
+        SimpleTextUtil.write(output, Float.toString(valueFloat), scratch);
+        break;
+      case FLOAT_64:
+        double valueDouble = Double.longBitsToDouble(DocValuesArraySource
+            .asLong(spare));
+        SimpleTextUtil.write(output, Double.toString(valueDouble), scratch);
+        break;
+      default:
+        throw new IllegalArgumentException("unsupported type: " + type);
+      }
+    } else {
+      switch (type) {
+      case BYTES_FIXED_DEREF:
+      case BYTES_FIXED_SORTED:
+      case BYTES_FIXED_STRAIGHT:
+        if(zeroBytes == null) {
+          assert fixedSize > 0;
+          zeroBytes = new BytesRef(new byte[fixedSize]);
+        }
+        SimpleTextUtil.write(output, zeroBytes);
+        break;
+      case BYTES_VAR_DEREF:
+      case BYTES_VAR_SORTED:
+      case BYTES_VAR_STRAIGHT:
+        scratch.length = 0;
+        SimpleTextUtil.write(output, scratch);
+        break;
+      case FIXED_INTS_16:
+      case FIXED_INTS_32:
+      case FIXED_INTS_64:
+      case FIXED_INTS_8:
+      case VAR_INTS:
+        SimpleTextUtil.write(output, ZERO_INT);
+        break;
+      case FLOAT_32:
+      case FLOAT_64:
+        SimpleTextUtil.write(output, ZERO_DOUBLE);
+        break;
+      default:
+        throw new IllegalArgumentException("unsupported type: " + type);
+      }
+    }
+
+  }
+
+  @Override
+  protected Type getType() {
+    return type;
+  }
+  
+  
+
+}
\ No newline at end of file

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java?rev=1297920&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java Wed Mar  7 10:26:44 2012
@@ -0,0 +1,53 @@
+package org.apache.lucene.codecs.simpletext;
+
+/**
+ * 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 java.util.Set;
+
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.codecs.PerDocProducer;
+import org.apache.lucene.index.PerDocWriteState;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.util.BytesRef;
+/**
+ * @lucene.experimental
+ */
+public class SimpleTextDocValuesFormat extends DocValuesFormat {
+  private static final String DOC_VALUES_SEG_SUFFIX = "dv";
+  @Override
+  public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
+    return new SimpleTextPerDocConsumer(state, DOC_VALUES_SEG_SUFFIX);
+  }
+
+  @Override
+  public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
+    return new SimpleTextPerDocProducer(state, BytesRef.getUTF8SortedAsUnicodeComparator(), DOC_VALUES_SEG_SUFFIX);
+  }
+
+  static String docValuesId(String segmentsName, int fieldId) {
+    return segmentsName + "_" + fieldId;
+  }
+
+  @Override
+  public void files(SegmentInfo info, Set<String> files)
+      throws IOException {
+    SimpleTextPerDocConsumer.files(info, files, DOC_VALUES_SEG_SUFFIX);
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java?rev=1297920&r1=1297919&r2=1297920&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java Wed Mar  7 10:26:44 2012
@@ -18,35 +18,123 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
+import java.util.Comparator;
+import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PerDocConsumer;
 import org.apache.lucene.codecs.PerDocProducer;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 
 /**
  * plain-text norms format
  * <p>
  * <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
+ * 
  * @lucene.experimental
  */
 public class SimpleTextNormsFormat extends NormsFormat {
+  private static final String NORMS_SEG_SUFFIX = "len";
   
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new SimpleTextNormsConsumer(state.directory, state.segmentName, state.context);
+    return new SimpleTextNormsPerDocConsumer(state, NORMS_SEG_SUFFIX);
   }
-
+  
   @Override
   public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
-    return new SimpleTextNormsProducer(state.dir, state.segmentInfo, state.fieldInfos, state.context);
+    return new SimpleTextNormsPerDocProducer(state,
+        BytesRef.getUTF8SortedAsUnicodeComparator(), NORMS_SEG_SUFFIX);
   }
-
+  
   @Override
   public void files(SegmentInfo info, Set<String> files) throws IOException {
-    SimpleTextNormsConsumer.files(info, files);
-  }   
+    SimpleTextNormsPerDocConsumer.files(info, files);
+  }
+  
+  public static class SimpleTextNormsPerDocProducer extends
+      SimpleTextPerDocProducer {
+    
+    public SimpleTextNormsPerDocProducer(SegmentReadState state,
+        Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
+      super(state, comp, segmentSuffix);
+    }
+    
+    @Override
+    protected boolean canLoad(FieldInfo info) {
+      return info.normsPresent();
+    }
+    
+    @Override
+    protected Type getDocValuesType(FieldInfo info) {
+      return info.getNormType();
+    }
+    
+    @Override
+    protected boolean anyDocValuesFields(FieldInfos infos) {
+      return infos.hasNorms();
+    }
+    
+  }
+  
+  public static class SimpleTextNormsPerDocConsumer extends
+      SimpleTextPerDocConsumer {
+    
+    public SimpleTextNormsPerDocConsumer(PerDocWriteState state,
+        String segmentSuffix) throws IOException {
+      super(state, segmentSuffix);
+    }
+    
+    @Override
+    protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
+        throws IOException {
+      return reader.normValues(info.name);
+    }
+    
+    @Override
+    protected boolean canMerge(FieldInfo info) {
+      return info.normsPresent();
+    }
+    
+    @Override
+    protected Type getDocValuesType(FieldInfo info) {
+      return info.getNormType();
+    }
+    
+    @Override
+    public void abort() {
+      Set<String> files = new HashSet<String>();
+      filesInternal(state.fieldInfos, state.segmentName, files, segmentSuffix);
+      IOUtils.deleteFilesIgnoringExceptions(state.directory,
+          files.toArray(new String[0]));
+    }
+    
+    public static void files(SegmentInfo segmentInfo, Set<String> files)
+        throws IOException {
+      filesInternal(segmentInfo.getFieldInfos(), segmentInfo.name, files,
+          NORMS_SEG_SUFFIX);
+    }
+    
+    public static void filesInternal(FieldInfos fieldInfos, String segmentName,
+        Set<String> files, String segmentSuffix) {
+      for (FieldInfo fieldInfo : fieldInfos) {
+        if (fieldInfo.normsPresent()) {
+          String id = docValuesId(segmentName, fieldInfo.number);
+          files.add(IndexFileNames.segmentFileName(id, "",
+              segmentSuffix));
+        }
+      }
+    }
+  }
 }

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java?rev=1297920&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java Wed Mar  7 10:26:44 2012
@@ -0,0 +1,94 @@
+package org.apache.lucene.codecs.simpletext;
+/**
+ * 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 java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.PerDocConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.PerDocWriteState;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * @lucene.experimental
+ */
+class SimpleTextPerDocConsumer extends PerDocConsumer {
+
+  protected final PerDocWriteState state;
+  protected final String segmentSuffix;
+  public SimpleTextPerDocConsumer(PerDocWriteState state, String segmentSuffix)
+      throws IOException {
+    this.state = state;
+    this.segmentSuffix = segmentSuffix;
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+
+  @Override
+  public DocValuesConsumer addValuesField(Type type, FieldInfo field)
+      throws IOException {
+    return new SimpleTextDocValuesConsumer(SimpleTextDocValuesFormat.docValuesId(state.segmentName,
+        field.number), state.directory, state.context, type, segmentSuffix);
+  }
+
+  @Override
+  public void abort() {
+    Set<String> files = new HashSet<String>();
+    files(state.directory, state.fieldInfos, state.segmentName, files, segmentSuffix);
+    IOUtils.deleteFilesIgnoringExceptions(state.directory,
+        files.toArray(new String[0]));
+  }
+  
+  
+  static void files(SegmentInfo info, Set<String> files, String segmentSuffix) throws IOException {
+    files(info.dir, info.getFieldInfos(), info.name, files, segmentSuffix);
+  }
+  
+  static String docValuesId(String segmentsName, int fieldId) {
+    return segmentsName + "_" + fieldId;
+  }
+
+  @SuppressWarnings("fallthrough")
+  private static void files(Directory dir, FieldInfos fieldInfos,
+      String segmentName, Set<String> files, String segmentSuffix) {
+    for (FieldInfo fieldInfo : fieldInfos) {
+      if (fieldInfo.hasDocValues()) {
+        String filename = docValuesId(segmentName, fieldInfo.number);
+        files.add(IndexFileNames.segmentFileName(filename, "",
+            segmentSuffix));
+        try {
+          assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
+              segmentSuffix));
+        } catch (IOException e) {
+          // don't throw checked exception - dir is only used in assert
+          throw new RuntimeException(e);
+        }
+      }
+    }
+  }
+
+}
\ No newline at end of file