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/02/07 21:48:28 UTC

svn commit: r1443717 [2/14] - in /lucene/dev/trunk: ./ dev-tools/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/icu/src/java/org/apache/lucene/collation/ lucene/analysis/icu/src/test/org/apache/lucene/collation/ lucene/backwards/ luc...

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java Thu Feb  7 20:48:21 2013
@@ -27,8 +27,8 @@ import org.apache.lucene.index.CorruptIn
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -39,8 +39,10 @@ import org.apache.lucene.util.IOUtils;
  * 
  * @lucene.experimental
  * @see Lucene40FieldInfosFormat
+ * @deprecated Only for reading old 4.0 and 4.1 segments
  */
-public class Lucene40FieldInfosReader extends FieldInfosReader {
+@Deprecated
+class Lucene40FieldInfosReader extends FieldInfosReader {
 
   /** Sole constructor. */
   public Lucene40FieldInfosReader() {
@@ -48,14 +50,14 @@ public class Lucene40FieldInfosReader ex
 
   @Override
   public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosWriter.FIELD_INFOS_EXTENSION);
+    final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosFormat.FIELD_INFOS_EXTENSION);
     IndexInput input = directory.openInput(fileName, iocontext);
     
     boolean success = false;
     try {
-      CodecUtil.checkHeader(input, Lucene40FieldInfosWriter.CODEC_NAME, 
-                                   Lucene40FieldInfosWriter.FORMAT_START, 
-                                   Lucene40FieldInfosWriter.FORMAT_CURRENT);
+      CodecUtil.checkHeader(input, Lucene40FieldInfosFormat.CODEC_NAME, 
+                                   Lucene40FieldInfosFormat.FORMAT_START, 
+                                   Lucene40FieldInfosFormat.FORMAT_CURRENT);
 
       final int size = input.readVInt(); //read in the size
       FieldInfo infos[] = new FieldInfo[size];
@@ -64,18 +66,18 @@ public class Lucene40FieldInfosReader ex
         String name = input.readString();
         final int fieldNumber = input.readVInt();
         byte bits = input.readByte();
-        boolean isIndexed = (bits & Lucene40FieldInfosWriter.IS_INDEXED) != 0;
-        boolean storeTermVector = (bits & Lucene40FieldInfosWriter.STORE_TERMVECTOR) != 0;
-        boolean omitNorms = (bits & Lucene40FieldInfosWriter.OMIT_NORMS) != 0;
-        boolean storePayloads = (bits & Lucene40FieldInfosWriter.STORE_PAYLOADS) != 0;
+        boolean isIndexed = (bits & Lucene40FieldInfosFormat.IS_INDEXED) != 0;
+        boolean storeTermVector = (bits & Lucene40FieldInfosFormat.STORE_TERMVECTOR) != 0;
+        boolean omitNorms = (bits & Lucene40FieldInfosFormat.OMIT_NORMS) != 0;
+        boolean storePayloads = (bits & Lucene40FieldInfosFormat.STORE_PAYLOADS) != 0;
         final IndexOptions indexOptions;
         if (!isIndexed) {
           indexOptions = null;
-        } else if ((bits & Lucene40FieldInfosWriter.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
+        } else if ((bits & Lucene40FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
           indexOptions = IndexOptions.DOCS_ONLY;
-        } else if ((bits & Lucene40FieldInfosWriter.OMIT_POSITIONS) != 0) {
+        } else if ((bits & Lucene40FieldInfosFormat.OMIT_POSITIONS) != 0) {
           indexOptions = IndexOptions.DOCS_AND_FREQS;
-        } else if ((bits & Lucene40FieldInfosWriter.STORE_OFFSETS_IN_POSTINGS) != 0) {
+        } else if ((bits & Lucene40FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS) != 0) {
           indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
         } else {
           indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
@@ -89,11 +91,20 @@ public class Lucene40FieldInfosReader ex
         }
         // DV Types are packed in one byte
         byte val = input.readByte();
-        final DocValues.Type docValuesType = getDocValuesType((byte) (val & 0x0F));
-        final DocValues.Type normsType = getDocValuesType((byte) ((val >>> 4) & 0x0F));
-        final Map<String,String> attributes = input.readStringStringMap();
+        final LegacyDocValuesType oldValuesType = getDocValuesType((byte) (val & 0x0F));
+        final LegacyDocValuesType oldNormsType = getDocValuesType((byte) ((val >>> 4) & 0x0F));
+        final Map<String,String> attributes = input.readStringStringMap();;
+        if (oldValuesType.mapping != null) {
+          attributes.put(LEGACY_DV_TYPE_KEY, oldValuesType.name());
+        }
+        if (oldNormsType.mapping != null) {
+          if (oldNormsType.mapping != DocValuesType.NUMERIC) {
+            throw new CorruptIndexException("invalid norm type: " + oldNormsType);
+          }
+          attributes.put(LEGACY_NORM_TYPE_KEY, oldNormsType.name());
+        }
         infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
-          omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes));
+          omitNorms, storePayloads, indexOptions, oldValuesType.mapping, oldNormsType.mapping, Collections.unmodifiableMap(attributes));
       }
 
       if (input.getFilePointer() != input.length()) {
@@ -110,39 +121,35 @@ public class Lucene40FieldInfosReader ex
       }
     }
   }
-
-  private static DocValues.Type getDocValuesType(final byte b) {
-    switch(b) {
-      case 0:
-        return null;
-      case 1:
-        return DocValues.Type.VAR_INTS;
-      case 2:
-        return DocValues.Type.FLOAT_32;
-      case 3:
-        return DocValues.Type.FLOAT_64;
-      case 4:
-        return DocValues.Type.BYTES_FIXED_STRAIGHT;
-      case 5:
-        return DocValues.Type.BYTES_FIXED_DEREF;
-      case 6:
-        return DocValues.Type.BYTES_VAR_STRAIGHT;
-      case 7:
-        return DocValues.Type.BYTES_VAR_DEREF;
-      case 8:
-        return DocValues.Type.FIXED_INTS_16;
-      case 9:
-        return DocValues.Type.FIXED_INTS_32;
-      case 10:
-        return DocValues.Type.FIXED_INTS_64;
-      case 11:
-        return DocValues.Type.FIXED_INTS_8;
-      case 12:
-        return DocValues.Type.BYTES_FIXED_SORTED;
-      case 13:
-        return DocValues.Type.BYTES_VAR_SORTED;
-      default:
-        throw new IllegalStateException("unhandled indexValues type " + b);
+  
+  static final String LEGACY_DV_TYPE_KEY = Lucene40FieldInfosReader.class.getSimpleName() + ".dvtype";
+  static final String LEGACY_NORM_TYPE_KEY = Lucene40FieldInfosReader.class.getSimpleName() + ".normtype";
+  
+  // mapping of 4.0 types -> 4.2 types
+  static enum LegacyDocValuesType {
+    NONE(null),
+    VAR_INTS(DocValuesType.NUMERIC),
+    FLOAT_32(DocValuesType.NUMERIC),
+    FLOAT_64(DocValuesType.NUMERIC),
+    BYTES_FIXED_STRAIGHT(DocValuesType.BINARY),
+    BYTES_FIXED_DEREF(DocValuesType.BINARY),
+    BYTES_VAR_STRAIGHT(DocValuesType.BINARY),
+    BYTES_VAR_DEREF(DocValuesType.BINARY),
+    FIXED_INTS_16(DocValuesType.NUMERIC),
+    FIXED_INTS_32(DocValuesType.NUMERIC),
+    FIXED_INTS_64(DocValuesType.NUMERIC),
+    FIXED_INTS_8(DocValuesType.NUMERIC),
+    BYTES_FIXED_SORTED(DocValuesType.SORTED),
+    BYTES_VAR_SORTED(DocValuesType.SORTED);
+    
+    final DocValuesType mapping;
+    LegacyDocValuesType(DocValuesType mapping) {
+      this.mapping = mapping;
     }
   }
+  
+  // decodes a 4.0 type
+  private static LegacyDocValuesType getDocValuesType(byte b) {
+    return LegacyDocValuesType.values()[b];
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java Thu Feb  7 20:48:21 2013
@@ -42,7 +42,7 @@ public final class Lucene40PostingsBaseF
 
   @Override
   public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
-    return new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+    return new Lucene40PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java Thu Feb  7 20:48:21 2013
@@ -248,12 +248,12 @@ public class Lucene40PostingsFormat exte
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+    PostingsReaderBase postings = new Lucene40PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
 
     boolean success = false;
     try {
       FieldsProducer ret = new BlockTreeTermsReader(
-                                                    state.dir,
+                                                    state.directory,
                                                     state.fieldInfos,
                                                     state.segmentInfo,
                                                     postings,

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html Thu Feb  7 20:48:21 2013
@@ -363,11 +363,11 @@ file, previously they were stored in tex
 frequencies.</li>
 <li>In version 4.0, the format of the inverted index became extensible via
 the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
-({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
-factors need no longer be a single byte, they can be any DocValues 
-{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
-strings, they can be any byte sequence. Term offsets can optionally be indexed 
-into the postings lists. Payloads can be stored in the term vectors.</li>
+({@code DocValues}) was introduced. Normalization factors need no longer be a 
+single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. 
+Terms need not be unicode strings, they can be any byte sequence. Term offsets 
+can optionally be indexed into the postings lists. Payloads can be stored in the 
+term vectors.</li>
 </ul>
 <a name="Limitations" id="Limitations"></a>
 <h2>Limitations</h2>

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java Thu Feb  7 20:48:21 2013
@@ -20,19 +20,19 @@ package org.apache.lucene.codecs.lucene4
 import java.io.IOException;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
 import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
 import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
 import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
@@ -64,9 +64,7 @@ public class Lucene41Codec extends Codec
   };
   private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
   private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
-  private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
   private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
-  private final NormsFormat normsFormat = new Lucene40NormsFormat();
   private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
   
   private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@@ -93,17 +91,12 @@ public class Lucene41Codec extends Codec
   }
 
   @Override
-  public final DocValuesFormat docValuesFormat() {
-    return docValuesFormat;
-  }
-
-  @Override
   public final PostingsFormat postingsFormat() {
     return postingsFormat;
   }
   
   @Override
-  public final FieldInfosFormat fieldInfosFormat() {
+  public FieldInfosFormat fieldInfosFormat() {
     return fieldInfosFormat;
   }
   
@@ -111,11 +104,6 @@ public class Lucene41Codec extends Codec
   public final SegmentInfoFormat segmentInfoFormat() {
     return infosFormat;
   }
-
-  @Override
-  public final NormsFormat normsFormat() {
-    return normsFormat;
-  }
   
   @Override
   public final LiveDocsFormat liveDocsFormat() {
@@ -131,5 +119,17 @@ public class Lucene41Codec extends Codec
     return defaultFormat;
   }
   
+  @Override
+  public DocValuesFormat docValuesFormat() {
+    return dvFormat;
+  }
+
   private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
+  private final DocValuesFormat dvFormat = new Lucene40DocValuesFormat();
+  private final NormsFormat normsFormat = new Lucene40NormsFormat();
+
+  @Override
+  public NormsFormat normsFormat() {
+    return normsFormat;
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java Thu Feb  7 20:48:21 2013
@@ -41,7 +41,7 @@ public final class Lucene41PostingsBaseF
 
   @Override
   public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
-    return new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+    return new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java Thu Feb  7 20:48:21 2013
@@ -427,14 +427,14 @@ public final class Lucene41PostingsForma
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.dir,
+    PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
                                                                 state.fieldInfos,
                                                                 state.segmentInfo,
                                                                 state.context,
                                                                 state.segmentSuffix);
     boolean success = false;
     try {
-      FieldsProducer ret = new BlockTreeTermsReader(state.dir,
+      FieldsProducer ret = new BlockTreeTermsReader(state.directory,
                                                     state.fieldInfos,
                                                     state.segmentInfo,
                                                     postingsReader,

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html Thu Feb  7 20:48:21 2013
@@ -368,11 +368,11 @@ file, previously they were stored in tex
 frequencies.</li>
 <li>In version 4.0, the format of the inverted index became extensible via
 the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
-({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
-factors need no longer be a single byte, they can be any DocValues 
-{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
-strings, they can be any byte sequence. Term offsets can optionally be indexed 
-into the postings lists. Payloads can be stored in the term vectors.</li>
+({@code DocValues}) was introduced. Normalization factors need no longer be a 
+single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}. 
+Terms need not be unicode strings, they can be any byte sequence. Term offsets 
+can optionally be indexed into the postings lists. Payloads can be stored in the 
+term vectors.</li>
 <li>In version 4.1, the format of the postings list changed to use either
 of FOR compression or variable-byte encoding, depending upon the frequency
 of the term.</li>

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Thu Feb  7 20:48:21 2013
@@ -37,7 +37,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.util.IOUtils;
 
 /**
- * Enables per field format support.
+ * Enables per field postings support.
  * <p>
  * Note, when extending this class, the name ({@link #getName}) is 
  * written into the index. In order for the field to be read, the

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java Thu Feb  7 20:48:21 2013
@@ -17,34 +17,21 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.AtomicReader; // javadocs
+import org.apache.lucene.search.FieldCache; // javadocs
 
 /**
+ * Syntactic sugar for encoding doubles as NumericDocValues
+ * via {@link Double#doubleToRawLongBits(double)}.
  * <p>
- * Field that stores a per-document <code>double</code> value for scoring, 
- * sorting or value retrieval. Here's an example usage:
- * 
- * <pre class="prettyprint">
- *   document.add(new DoubleDocValuesField(name, 22.0));
- * </pre>
- * 
+ * Per-document double values can be retrieved via
+ * {@link FieldCache#getDoubles(AtomicReader, String, boolean)}.
  * <p>
- * If you also need to store the value, you should add a
- * separate {@link StoredField} instance.
- * 
- * @see DocValues
- * */
-
-public class DoubleDocValuesField extends StoredField {
-
-  /**
-   * Type for 64-bit double DocValues.
-   */
-  public static final FieldType TYPE = new FieldType();
-  static {
-    TYPE.setDocValueType(DocValues.Type.FLOAT_64);
-    TYPE.freeze();
-  }
+ * <b>NOTE</b>: In most all cases this will be rather inefficient,
+ * requiring eight bytes per document. Consider encoding double
+ * values yourself with only as much precision as you require.
+ */
+public class DoubleDocValuesField extends NumericDocValuesField {
 
   /** 
    * Creates a new DocValues field with the specified 64-bit double value 
@@ -53,7 +40,16 @@ public class DoubleDocValuesField extend
    * @throws IllegalArgumentException if the field name is null
    */
   public DoubleDocValuesField(String name, double value) {
-    super(name, TYPE);
-    fieldsData = Double.valueOf(value);
+    super(name, Double.doubleToRawLongBits(value));
+  }
+
+  @Override
+  public void setDoubleValue(double value) {
+    super.setLongValue(Double.doubleToRawLongBits(value));
+  }
+  
+  @Override
+  public void setLongValue(long value) {
+    throw new IllegalArgumentException("cannot change value type from Double to Long");
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Field.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Field.java Thu Feb  7 20:48:21 2013
@@ -29,7 +29,6 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.index.IndexWriter; // javadocs
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.IndexableFieldType;
-import org.apache.lucene.index.Norm; // javadocs
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.FieldInvertState; // javadocs
@@ -38,13 +37,8 @@ import org.apache.lucene.index.FieldInve
  * Expert: directly create a field for a document.  Most
  * users should use one of the sugar subclasses: {@link
  * IntField}, {@link LongField}, {@link FloatField}, {@link
- * DoubleField}, {@link ByteDocValuesField}, {@link
- * ShortDocValuesField}, {@link IntDocValuesField}, {@link
- * LongDocValuesField}, {@link PackedLongDocValuesField},
- * {@link FloatDocValuesField}, {@link
- * DoubleDocValuesField}, {@link SortedBytesDocValuesField},
- * {@link DerefBytesDocValuesField}, {@link
- * StraightBytesDocValuesField}, {@link
+ * DoubleField}, {@link BinaryDocValuesField}, {@link
+ * NumericDocValuesField}, {@link SortedDocValuesField}, {@link
  * StringField}, {@link TextField}, {@link StoredField}.
  *
  * <p/> A field is a section of a Document. Each field has three

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java Thu Feb  7 20:48:21 2013
@@ -18,7 +18,7 @@ package org.apache.lucene.document;
  */
 
 import org.apache.lucene.analysis.Analyzer; // javadocs
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexableFieldType;
 import org.apache.lucene.search.NumericRangeQuery; // javadocs
@@ -55,7 +55,7 @@ public class FieldType implements Indexa
   private NumericType numericType;
   private boolean frozen;
   private int numericPrecisionStep = NumericUtils.PRECISION_STEP_DEFAULT;
-  private DocValues.Type docValueType;
+  private DocValuesType docValueType;
 
   /**
    * Create a new mutable FieldType with all of the properties from <code>ref</code>
@@ -416,21 +416,21 @@ public class FieldType implements Indexa
    * {@inheritDoc}
    * <p>
    * The default is <code>null</code> (no docValues) 
-   * @see #setDocValueType(DocValues.Type)
+   * @see #setDocValueType(org.apache.lucene.index.FieldInfo.DocValuesType)
    */
   @Override
-  public DocValues.Type docValueType() {
+  public DocValuesType docValueType() {
     return docValueType;
   }
 
   /**
-   * Set's the field's DocValues.Type
+   * Set's the field's DocValuesType
    * @param type DocValues type, or null if no DocValues should be stored.
    * @throws IllegalStateException if this FieldType is frozen against
    *         future modifications.
    * @see #docValueType()
    */
-  public void setDocValueType(DocValues.Type type) {
+  public void setDocValueType(DocValuesType type) {
     checkIfFrozen();
     docValueType = type;
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java Thu Feb  7 20:48:21 2013
@@ -17,33 +17,21 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.AtomicReader; // javadocs
+import org.apache.lucene.search.FieldCache; // javadocs
 
 /**
+ * Syntactic sugar for encoding floats as NumericDocValues
+ * via {@link Float#floatToRawIntBits(float)}.
  * <p>
- * Field that stores a per-document <code>float</code> value for scoring, 
- * sorting or value retrieval. Here's an example usage:
- * 
- * <pre class="prettyprint">
- *   document.add(new FloatDocValuesField(name, 22f));
- * </pre>
- * 
+ * Per-document floating point values can be retrieved via
+ * {@link FieldCache#getFloats(AtomicReader, String, boolean)}.
  * <p>
- * If you also need to store the value, you should add a
- * separate {@link StoredField} instance.
- * @see DocValues
- * */
-
-public class FloatDocValuesField extends StoredField {
-
-  /**
-   * Type for 32-bit float DocValues.
-   */
-  public static final FieldType TYPE = new FieldType();
-  static {
-    TYPE.setDocValueType(DocValues.Type.FLOAT_32);
-    TYPE.freeze();
-  }
+ * <b>NOTE</b>: In most all cases this will be rather inefficient,
+ * requiring four bytes per document. Consider encoding floating
+ * point values yourself with only as much precision as you require.
+ */
+public class FloatDocValuesField extends NumericDocValuesField {
 
   /** 
    * Creates a new DocValues field with the specified 32-bit float value 
@@ -52,7 +40,16 @@ public class FloatDocValuesField extends
    * @throws IllegalArgumentException if the field name is null
    */
   public FloatDocValuesField(String name, float value) {
-    super(name, TYPE);
-    fieldsData = Float.valueOf(value);
+    super(name, Float.floatToRawIntBits(value));
+  }
+
+  @Override
+  public void setFloatValue(float value) {
+    super.setLongValue(Float.floatToRawIntBits(value));
+  }
+  
+  @Override
+  public void setLongValue(long value) {
+    throw new IllegalArgumentException("cannot change value type from Float to Long");
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Thu Feb  7 20:48:21 2013
@@ -156,19 +156,30 @@ public abstract class AtomicReader exten
     }
     return null;
   }
-  
-  /**
-   * Returns {@link DocValues} for this field.
-   * This method may return null if the reader has no per-document
-   * values stored.
-   */
-  public abstract DocValues docValues(String field) throws IOException;
-  
-  /**
-   * Returns {@link DocValues} for this field's normalization values.
-   * This method may return null if the field has no norms.
-   */
-  public abstract DocValues normValues(String field) throws IOException;
+
+  /** Returns {@link NumericDocValues} for this field, or
+   *  null if no {@link NumericDocValues} were indexed for
+   *  this field.  The returned instance should only be
+   *  used by a single thread. */
+  public abstract NumericDocValues getNumericDocValues(String field) throws IOException;
+
+  /** Returns {@link BinaryDocValues} for this field, or
+   *  null if no {@link BinaryDocValues} were indexed for
+   *  this field.  The returned instance should only be
+   *  used by a single thread. */
+  public abstract BinaryDocValues getBinaryDocValues(String field) throws IOException;
+
+  /** Returns {@link SortedDocValues} for this field, or
+   *  null if no {@link SortedDocValues} were indexed for
+   *  this field.  The returned instance should only be
+   *  used by a single thread. */
+  public abstract SortedDocValues getSortedDocValues(String field) throws IOException;
+
+  /** Returns {@link NumericDocValues} representing norms
+   *  for this field, or null if no {@link NumericDocValues}
+   *  were indexed. The returned instance should only be
+   *  used by a single thread. */
+  public abstract NumericDocValues getNormValues(String field) throws IOException;
 
   /**
    * Get the {@link FieldInfos} describing all fields in

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Thu Feb  7 20:48:21 2013
@@ -31,10 +31,7 @@ import java.util.Map;
 import org.apache.lucene.codecs.BlockTreeTermsReader;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat; // javadocs
-import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldType; // for javadocs
-import org.apache.lucene.index.DocValues.SortedSource;
-import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
@@ -183,7 +180,7 @@ public class CheckIndex {
       /** Number of deleted documents. */
       public int numDeleted;
 
-      /** True if we were able to open a SegmentReader on this
+      /** True if we were able to open an AtomicReader on this
        *  segment. */
       public boolean openReaderPassed;
 
@@ -343,9 +340,9 @@ public class CheckIndex {
     setInfoStream(out, false);
   }
 
-  private void msg(String msg) {
-    if (infoStream != null)
-      infoStream.println(msg);
+  private static void msg(PrintStream out, String msg) {
+    if (out != null)
+      out.println(msg);
   }
 
   /** Returns a {@link Status} instance detailing
@@ -381,7 +378,7 @@ public class CheckIndex {
     try {
       sis.read(dir);
     } catch (Throwable t) {
-      msg("ERROR: could not read any segments file in directory");
+      msg(infoStream, "ERROR: could not read any segments file in directory");
       result.missingSegments = true;
       if (infoStream != null)
         t.printStackTrace(infoStream);
@@ -416,7 +413,7 @@ public class CheckIndex {
     try {
       input = dir.openInput(segmentsFileName, IOContext.DEFAULT);
     } catch (Throwable t) {
-      msg("ERROR: could not open segments file in directory");
+      msg(infoStream, "ERROR: could not open segments file in directory");
       if (infoStream != null)
         t.printStackTrace(infoStream);
       result.cantOpenSegments = true;
@@ -426,7 +423,7 @@ public class CheckIndex {
     try {
       format = input.readInt();
     } catch (Throwable t) {
-      msg("ERROR: could not read segment file version in directory");
+      msg(infoStream, "ERROR: could not read segment file version in directory");
       if (infoStream != null)
         t.printStackTrace(infoStream);
       result.missingSegmentVersion = true;
@@ -460,7 +457,7 @@ public class CheckIndex {
       versionString = oldest.equals(newest) ? ( "version=" + oldest ) : ("versions=[" + oldest + " .. " + newest + "]");
     }
 
-    msg("Segments file=" + segmentsFileName + " numSegments=" + numSegments
+    msg(infoStream, "Segments file=" + segmentsFileName + " numSegments=" + numSegments
         + " " + versionString + " format=" + sFormat + userDataString);
 
     if (onlySegments != null) {
@@ -472,11 +469,11 @@ public class CheckIndex {
           infoStream.print(" " + s);
       }
       result.segmentsChecked.addAll(onlySegments);
-      msg(":");
+      msg(infoStream, ":");
     }
 
     if (skip) {
-      msg("\nERROR: this index appears to be created by a newer version of Lucene than this tool was compiled on; please re-compile this tool on the matching version of Lucene; exiting");
+      msg(infoStream, "\nERROR: this index appears to be created by a newer version of Lucene than this tool was compiled on; please re-compile this tool on the matching version of Lucene; exiting");
       result.toolOutOfDate = true;
       return result;
     }
@@ -497,38 +494,41 @@ public class CheckIndex {
       }
       Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
       result.segmentInfos.add(segInfoStat);
-      msg("  " + (1+i) + " of " + numSegments + ": name=" + info.info.name + " docCount=" + info.info.getDocCount());
+      msg(infoStream, "  " + (1+i) + " of " + numSegments + ": name=" + info.info.name + " docCount=" + info.info.getDocCount());
       segInfoStat.name = info.info.name;
       segInfoStat.docCount = info.info.getDocCount();
 
       int toLoseDocCount = info.info.getDocCount();
 
-      SegmentReader reader = null;
+      AtomicReader reader = null;
 
       try {
         final Codec codec = info.info.getCodec();
-        msg("    codec=" + codec);
+        msg(infoStream, "    codec=" + codec);
         segInfoStat.codec = codec;
-        msg("    compound=" + info.info.getUseCompoundFile());
+        msg(infoStream, "    compound=" + info.info.getUseCompoundFile());
         segInfoStat.compound = info.info.getUseCompoundFile();
-        msg("    numFiles=" + info.files().size());
+        msg(infoStream, "    numFiles=" + info.files().size());
         segInfoStat.numFiles = info.files().size();
         segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.);
-        msg("    size (MB)=" + nf.format(segInfoStat.sizeMB));
+        msg(infoStream, "    size (MB)=" + nf.format(segInfoStat.sizeMB));
         Map<String,String> diagnostics = info.info.getDiagnostics();
         segInfoStat.diagnostics = diagnostics;
         if (diagnostics.size() > 0) {
-          msg("    diagnostics = " + diagnostics);
+          msg(infoStream, "    diagnostics = " + diagnostics);
         }
 
-        // TODO: we could append the info attributes() to the msg?
+        Map<String,String> atts = info.info.attributes();
+        if (atts != null && !atts.isEmpty()) {
+          msg(infoStream, "    attributes = " + atts);
+        }
 
         if (!info.hasDeletions()) {
-          msg("    no deletions");
+          msg(infoStream, "    no deletions");
           segInfoStat.hasDeletions = false;
         }
         else{
-          msg("    has deletions [delGen=" + info.getDelGen() + "]");
+          msg(infoStream, "    has deletions [delGen=" + info.getDelGen() + "]");
           segInfoStat.hasDeletions = true;
           segInfoStat.deletionsGen = info.getDelGen();
         }
@@ -566,7 +566,7 @@ public class CheckIndex {
           }
           
           segInfoStat.numDeleted = info.info.getDocCount() - numDocs;
-          msg("OK [" + (segInfoStat.numDeleted) + " deleted docs]");
+          msg(infoStream, "OK [" + (segInfoStat.numDeleted) + " deleted docs]");
         } else {
           if (info.getDelCount() != 0) {
             throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - numDocs));
@@ -580,7 +580,7 @@ public class CheckIndex {
               }
             }
           }
-          msg("OK");
+          msg(infoStream, "OK");
         }
         if (reader.maxDoc() != info.info.getDocCount()) {
           throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.info.getDocCount());
@@ -591,22 +591,22 @@ public class CheckIndex {
           infoStream.print("    test: fields..............");
         }         
         FieldInfos fieldInfos = reader.getFieldInfos();
-        msg("OK [" + fieldInfos.size() + " fields]");
+        msg(infoStream, "OK [" + fieldInfos.size() + " fields]");
         segInfoStat.numFields = fieldInfos.size();
         
         // Test Field Norms
-        segInfoStat.fieldNormStatus = testFieldNorms(fieldInfos, reader);
+        segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream);
 
         // Test the Term Index
-        segInfoStat.termIndexStatus = testPostings(fieldInfos, reader);
+        segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose);
 
         // Test Stored Fields
-        segInfoStat.storedFieldStatus = testStoredFields(info, reader, nf);
+        segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream);
 
         // Test Term Vectors
-        segInfoStat.termVectorStatus = testTermVectors(fieldInfos, info, reader, nf);
-        
-        segInfoStat.docValuesStatus = testDocValues(info, fieldInfos, reader);
+        segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors);
+
+        segInfoStat.docValuesStatus = testDocValues(reader, infoStream);
 
         // Rethrow the first exception we encountered
         //  This will cause stats for failed segments to be incremented properly
@@ -622,16 +622,16 @@ public class CheckIndex {
           throw new RuntimeException("DocValues test failed");
         }
 
-        msg("");
+        msg(infoStream, "");
 
       } catch (Throwable t) {
-        msg("FAILED");
+        msg(infoStream, "FAILED");
         String comment;
         comment = "fixIndex() would remove reference to this segment";
-        msg("    WARNING: " + comment + "; full exception:");
+        msg(infoStream, "    WARNING: " + comment + "; full exception:");
         if (infoStream != null)
           t.printStackTrace(infoStream);
-        msg("");
+        msg(infoStream, "");
         result.totLoseDocCount += toLoseDocCount;
         result.numBadSegments++;
         continue;
@@ -647,16 +647,16 @@ public class CheckIndex {
     if (0 == result.numBadSegments) {
       result.clean = true;
     } else
-      msg("WARNING: " + result.numBadSegments + " broken segments (containing " + result.totLoseDocCount + " documents) detected");
+      msg(infoStream, "WARNING: " + result.numBadSegments + " broken segments (containing " + result.totLoseDocCount + " documents) detected");
 
     if ( ! (result.validCounter = (result.maxSegmentName < sis.counter))) {
       result.clean = false;
       result.newSegments.counter = result.maxSegmentName + 1; 
-      msg("ERROR: Next segment name counter " + sis.counter + " is not greater than max segment name " + result.maxSegmentName);
+      msg(infoStream, "ERROR: Next segment name counter " + sis.counter + " is not greater than max segment name " + result.maxSegmentName);
     }
     
     if (result.clean) {
-      msg("No problems were detected with this index.\n");
+      msg(infoStream, "No problems were detected with this index.\n");
     }
 
     return result;
@@ -664,8 +664,9 @@ public class CheckIndex {
 
   /**
    * Test field norms.
+   * @lucene.experimental
    */
-  private Status.FieldNormStatus testFieldNorms(FieldInfos fieldInfos, SegmentReader reader) {
+  public static Status.FieldNormStatus testFieldNorms(AtomicReader reader, PrintStream infoStream) {
     final Status.FieldNormStatus status = new Status.FieldNormStatus();
 
     try {
@@ -673,21 +674,20 @@ public class CheckIndex {
       if (infoStream != null) {
         infoStream.print("    test: field norms.........");
       }
-      for (FieldInfo info : fieldInfos) {
+      for (FieldInfo info : reader.getFieldInfos()) {
         if (info.hasNorms()) {
-          DocValues dv = reader.normValues(info.name);
-          checkDocValues(dv, info.name, info.getNormType(), reader.maxDoc());
+          checkNorms(info, reader, infoStream);
           ++status.totFields;
         } else {
-          if (reader.normValues(info.name) != null) {
+          if (reader.getNormValues(info.name) != null) {
             throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
           }
         }
       }
 
-      msg("OK [" + status.totFields + " fields]");
+      msg(infoStream, "OK [" + status.totFields + " fields]");
     } catch (Throwable e) {
-      msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
+      msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
       if (infoStream != null) {
         e.printStackTrace(infoStream);
@@ -701,14 +701,14 @@ public class CheckIndex {
    * checks Fields api is consistent with itself.
    * searcher is optional, to verify with queries. Can be null.
    */
-  private Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors) throws IOException {
+  private static Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors, PrintStream infoStream, boolean verbose) throws IOException {
     // TODO: we should probably return our own stats thing...?!
     
     final Status.TermIndexStatus status = new Status.TermIndexStatus();
     int computedFieldCount = 0;
     
     if (fields == null) {
-      msg("OK [no fields/terms]");
+      msg(infoStream, "OK [no fields/terms]");
       return status;
     }
     
@@ -769,7 +769,7 @@ public class CheckIndex {
           break;
         }
         
-        checkBounds(term);
+        assert term.isValid();
         
         // make sure terms arrive in order according to
         // the comp
@@ -859,7 +859,7 @@ public class CheckIndex {
               lastPos = pos;
               BytesRef payload = postings.getPayload();
               if (payload != null) {
-                checkBounds(payload);
+                assert payload.isValid();
               }
               if (payload != null && payload.length < 1) {
                 throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + " payload length is out of bounds " + payload.length);
@@ -1155,7 +1155,7 @@ public class CheckIndex {
     }
 
     if (doPrint) {
-      msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
+      msg(infoStream, "OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
     }
     
     if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) {
@@ -1170,8 +1170,17 @@ public class CheckIndex {
 
   /**
    * Test the term index.
+   * @lucene.experimental
+   */
+  public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream) {
+    return testPostings(reader, infoStream, false);
+  }
+  
+  /**
+   * Test the term index.
+   * @lucene.experimental
    */
-  private Status.TermIndexStatus testPostings(FieldInfos fieldInfos, SegmentReader reader) {
+  public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream, boolean verbose) {
 
     // TODO: we should go and verify term vectors match, if
     // crossCheckTermVectors is on...
@@ -1186,15 +1195,16 @@ public class CheckIndex {
       }
 
       final Fields fields = reader.fields();
-      status = checkFields(fields, liveDocs, maxDoc, fieldInfos, true, false);
+      final FieldInfos fieldInfos = reader.getFieldInfos();
+      status = checkFields(fields, liveDocs, maxDoc, fieldInfos, true, false, infoStream, verbose);
       if (liveDocs != null) {
         if (infoStream != null) {
           infoStream.print("    test (ignoring deletes): terms, freq, prox...");
         }
-        checkFields(fields, null, maxDoc, fieldInfos, true, false);
+        checkFields(fields, null, maxDoc, fieldInfos, true, false, infoStream, verbose);
       }
     } catch (Throwable e) {
-      msg("ERROR: " + e);
+      msg(infoStream, "ERROR: " + e);
       status = new Status.TermIndexStatus();
       status.error = e;
       if (infoStream != null) {
@@ -1206,9 +1216,10 @@ public class CheckIndex {
   }
   
   /**
-   * Test stored fields for a segment.
+   * Test stored fields.
+   * @lucene.experimental
    */
-  private Status.StoredFieldStatus testStoredFields(SegmentInfoPerCommit info, SegmentReader reader, NumberFormat format) {
+  public static Status.StoredFieldStatus testStoredFields(AtomicReader reader, PrintStream infoStream) {
     final Status.StoredFieldStatus status = new Status.StoredFieldStatus();
 
     try {
@@ -1218,7 +1229,7 @@ public class CheckIndex {
 
       // Scan stored fields for all documents
       final Bits liveDocs = reader.getLiveDocs();
-      for (int j = 0; j < info.info.getDocCount(); ++j) {
+      for (int j = 0; j < reader.maxDoc(); ++j) {
         // Intentionally pull even deleted documents to
         // make sure they too are not corrupt:
         StoredDocument doc = reader.document(j);
@@ -1233,10 +1244,10 @@ public class CheckIndex {
         throw new RuntimeException("docCount=" + status.docCount + " but saw " + status.docCount + " undeleted docs");
       }
 
-      msg("OK [" + status.totFields + " total field count; avg " + 
-          format.format((((float) status.totFields)/status.docCount)) + " fields per doc]");      
+      msg(infoStream, "OK [" + status.totFields + " total field count; avg " + 
+          NumberFormat.getInstance(Locale.ROOT).format((((float) status.totFields)/status.docCount)) + " fields per doc]");      
     } catch (Throwable e) {
-      msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
+      msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
       if (infoStream != null) {
         e.printStackTrace(infoStream);
@@ -1246,124 +1257,33 @@ public class CheckIndex {
     return status;
   }
   
-  /** Helper method to verify values (either docvalues or norms), also checking
-   *  type and size against fieldinfos/segmentinfo
+  /**
+   * Test docvalues.
+   * @lucene.experimental
    */
-  private void checkDocValues(DocValues docValues, String fieldName, DocValues.Type expectedType, int expectedDocs) throws IOException {
-    if (docValues == null) {
-      throw new RuntimeException("field: " + fieldName + " omits docvalues but should have them!");
-    }
-    DocValues.Type type = docValues.getType();
-    if (type != expectedType) {
-      throw new RuntimeException("field: " + fieldName + " has type: " + type + " but fieldInfos says:" + expectedType);
-    }
-    final Source values = docValues.getDirectSource();
-    int size = docValues.getValueSize();
-    for (int i = 0; i < expectedDocs; i++) {
-      switch (type) {
-      case BYTES_FIXED_SORTED:
-      case BYTES_VAR_SORTED:
-      case BYTES_FIXED_DEREF:
-      case BYTES_FIXED_STRAIGHT:
-      case BYTES_VAR_DEREF:
-      case BYTES_VAR_STRAIGHT:
-        BytesRef bytes = new BytesRef();
-        values.getBytes(i, bytes);
-        if (size != -1 && size != bytes.length) {
-          throw new RuntimeException("field: " + fieldName + " returned wrongly sized bytes, was: " + bytes.length + " should be: " + size);
-        }
-        break;
-      case FLOAT_32:
-        assert size == 4;
-        values.getFloat(i);
-        break;
-      case FLOAT_64:
-        assert size == 8;
-        values.getFloat(i);
-        break;
-      case VAR_INTS:
-        assert size == -1;
-        values.getInt(i);
-        break;
-      case FIXED_INTS_16:
-        assert size == 2;
-        values.getInt(i);
-        break;
-      case FIXED_INTS_32:
-        assert size == 4;
-        values.getInt(i);
-        break;
-      case FIXED_INTS_64:
-        assert size == 8;
-        values.getInt(i);
-        break;
-      case FIXED_INTS_8:
-        assert size == 1;
-        values.getInt(i);
-        break;
-      default:
-        throw new IllegalArgumentException("Field: " + fieldName
-                    + " - no such DocValues type: " + type);
-      }
-    }
-    if (type == DocValues.Type.BYTES_FIXED_SORTED || type == DocValues.Type.BYTES_VAR_SORTED) {
-      // check sorted bytes
-      SortedSource sortedValues = values.asSortedSource();
-      Comparator<BytesRef> comparator = sortedValues.getComparator();
-      int maxOrd = sortedValues.getValueCount() - 1;
-      FixedBitSet seenOrds = new FixedBitSet(sortedValues.getValueCount());
-      int lastOrd = -1;
-      BytesRef lastBytes = new BytesRef();
-      for (int i = 0; i < expectedDocs; i++) {
-        int ord = sortedValues.ord(i);
-        if (ord < 0 || ord > maxOrd) {
-          throw new RuntimeException("field: " + fieldName + " ord is out of bounds: " + ord);
-        }
-        BytesRef bytes = new BytesRef();
-        sortedValues.getByOrd(ord, bytes);
-        if (lastOrd != -1) {
-          int ordComp = Integer.signum(new Integer(ord).compareTo(new Integer(lastOrd)));
-          int bytesComp = Integer.signum(comparator.compare(bytes, lastBytes));
-          if (ordComp != bytesComp) {
-            throw new RuntimeException("field: " + fieldName + " ord comparison is wrong: " + ordComp + " comparator claims: " + bytesComp);
-          }
-        }
-        lastOrd = ord;
-        lastBytes = bytes;
-        seenOrds.set(ord);
-      }
-      if (seenOrds.cardinality() != sortedValues.getValueCount()) {
-        // TODO: find the bug here and figure out a workaround (we can implement in LUCENE-4547's back compat layer maybe)
-        // basically ord 0 is unused by any docs: so the sortedbytes ords are all off-by-one
-        // does it always happen? e.g. maybe only if there are missing values? or a bug in its merge optimizations?
-        // throw new RuntimeException("dv for field: " + fieldName + " has holes in its ords, valueCount=" + sortedValues.getValueCount() + " but only used: " + seenOrds.cardinality());
-      }
-    }
-  }
-  
-  private Status.DocValuesStatus testDocValues(SegmentInfoPerCommit info,
-                                               FieldInfos fieldInfos,
-                                               SegmentReader reader) {
+  public static Status.DocValuesStatus testDocValues(AtomicReader reader,
+                                                     PrintStream infoStream) {
     final Status.DocValuesStatus status = new Status.DocValuesStatus();
     try {
       if (infoStream != null) {
         infoStream.print("    test: docvalues...........");
       }
-      for (FieldInfo fieldInfo : fieldInfos) {
+      for (FieldInfo fieldInfo : reader.getFieldInfos()) {
         if (fieldInfo.hasDocValues()) {
           status.totalValueFields++;
-          final DocValues docValues = reader.docValues(fieldInfo.name);
-          checkDocValues(docValues, fieldInfo.name, fieldInfo.getDocValuesType(), reader.maxDoc());
+          checkDocValues(fieldInfo, reader, infoStream);
         } else {
-          if (reader.docValues(fieldInfo.name) != null) {
+          if (reader.getBinaryDocValues(fieldInfo.name) != null ||
+              reader.getNumericDocValues(fieldInfo.name) != null ||
+              reader.getSortedDocValues(fieldInfo.name) != null) {
             throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
           }
         }
       }
 
-      msg("OK [" + status.docCount + " total doc count; " + status.totalValueFields + " docvalues fields]");
+      msg(infoStream, "OK [" + status.docCount + " total doc count; " + status.totalValueFields + " docvalues fields]");
     } catch (Throwable e) {
-      msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
+      msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
       if (infoStream != null) {
         e.printStackTrace(infoStream);
@@ -1372,37 +1292,94 @@ public class CheckIndex {
     return status;
   }
   
-  // basic value checks
-  private static void checkBounds(BytesRef b) {
-    if (b.bytes == null) {
-      throw new RuntimeException("bytes is null");
-    }
-    if (b.length < 0) {
-      throw new RuntimeException("length is negative: " + b.length);
-    }
-    if (b.length > b.bytes.length) {
-      throw new RuntimeException("length is out of bounds: " + b.length + ", bytes.length=" + b.bytes.length);
+  private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv) {
+    BytesRef scratch = new BytesRef();
+    for (int i = 0; i < reader.maxDoc(); i++) {
+      dv.get(i, scratch);
+      assert scratch.isValid();
     }
-    if (b.offset < 0) {
-      throw new RuntimeException("offset is negative: " + b.offset);
+  }
+  
+  private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv) {
+    checkBinaryDocValues(fieldName, reader, dv);
+    final int maxOrd = dv.getValueCount()-1;
+    FixedBitSet seenOrds = new FixedBitSet(dv.getValueCount());
+    int maxOrd2 = -1;
+    for (int i = 0; i < reader.maxDoc(); i++) {
+      int ord = dv.getOrd(i);
+      if (ord < 0 || ord > maxOrd) {
+        throw new RuntimeException("ord out of bounds: " + ord);
+      }
+      maxOrd2 = Math.max(maxOrd2, ord);
+      seenOrds.set(ord);
+    }
+    if (maxOrd != maxOrd2) {
+      throw new RuntimeException("dv for field: " + fieldName + " reports wrong maxOrd=" + maxOrd + " but this is not the case: " + maxOrd2);
+    }
+    if (seenOrds.cardinality() != dv.getValueCount()) {
+      throw new RuntimeException("dv for field: " + fieldName + " has holes in its ords, valueCount=" + dv.getValueCount() + " but only used: " + seenOrds.cardinality());
+    }
+    BytesRef lastValue = null;
+    BytesRef scratch = new BytesRef();
+    for (int i = 0; i <= maxOrd; i++) {
+      dv.lookupOrd(i, scratch);
+      assert scratch.isValid();
+      if (lastValue != null) {
+        if (scratch.compareTo(lastValue) <= 0) {
+          throw new RuntimeException("dv for field: " + fieldName + " has ords out of order: " + lastValue + " >=" + scratch);
+        }
+      }
+      lastValue = BytesRef.deepCopyOf(scratch);
     }
-    if (b.offset > b.bytes.length) {
-      throw new RuntimeException("offset out of bounds: " + b.offset + ", length=" + b.length);
+  }
+  
+  private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv) {
+    for (int i = 0; i < reader.maxDoc(); i++) {
+      ndv.get(i);
     }
-    if (b.offset + b.length < 0) {
-      throw new RuntimeException("offset+length is negative: offset=" + b.offset + ",length=" + b.length);
+  }
+  
+  private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws Exception {
+    switch(fi.getDocValuesType()) {
+      case SORTED:
+        checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
+        break;
+      case BINARY:
+        checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
+        break;
+      case NUMERIC:
+        checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
+        break;
+      default:
+        throw new AssertionError();
     }
-    if (b.offset + b.length > b.bytes.length) {
-      throw new RuntimeException("offset+length out of bounds: offset=" + b.offset + ",length=" + b.length + ",bytes.length=" + b.bytes.length);
+  }
+  
+  private static void checkNorms(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws IOException {
+    switch(fi.getNormType()) {
+      case NUMERIC:
+        checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name));
+        break;
+      default:
+        throw new AssertionError("wtf: " + fi.getNormType());
     }
   }
 
   /**
-   * Test term vectors for a segment.
+   * Test term vectors.
+   * @lucene.experimental
    */
-  private Status.TermVectorStatus testTermVectors(FieldInfos fieldInfos, SegmentInfoPerCommit info, SegmentReader reader, NumberFormat format) {
-    final Status.TermVectorStatus status = new Status.TermVectorStatus();
+  public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream) {
+    return testTermVectors(reader, infoStream, false, false);
+  }
 
+  /**
+   * Test term vectors.
+   * @lucene.experimental
+   */
+  public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors) {
+    final Status.TermVectorStatus status = new Status.TermVectorStatus();
+    final FieldInfos fieldInfos = reader.getFieldInfos();
     final Bits onlyDocIsDeleted = new FixedBitSet(1);
     
     try {
@@ -1430,7 +1407,7 @@ public class CheckIndex {
       TermsEnum termsEnum = null;
       TermsEnum postingsTermsEnum = null;
 
-      for (int j = 0; j < info.info.getDocCount(); ++j) {
+      for (int j = 0; j < reader.maxDoc(); ++j) {
         // Intentionally pull/visit (but don't count in
         // stats) deleted documents to make sure they too
         // are not corrupt:
@@ -1441,10 +1418,10 @@ public class CheckIndex {
 
         if (tfv != null) {
           // First run with no deletions:
-          checkFields(tfv, null, 1, fieldInfos, false, true);
+          checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose);
 
           // Again, with the one doc deleted:
-          checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true);
+          checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true, infoStream, verbose);
 
           // Only agg stats if the doc is live:
           final boolean doStats = liveDocs == null || liveDocs.get(j);
@@ -1608,10 +1585,10 @@ public class CheckIndex {
         }
       }
       float vectorAvg = status.docCount == 0 ? 0 : status.totVectors / (float)status.docCount;
-      msg("OK [" + status.totVectors + " total vector count; avg " + 
-          format.format(vectorAvg) + " term/freq vector fields per doc]");
+      msg(infoStream, "OK [" + status.totVectors + " total vector count; avg " + 
+          NumberFormat.getInstance(Locale.ROOT).format(vectorAvg) + " term/freq vector fields per doc]");
     } catch (Throwable e) {
-      msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
+      msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
       if (infoStream != null) {
         e.printStackTrace(infoStream);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java Thu Feb  7 20:48:21 2013
@@ -24,6 +24,5 @@ abstract class DocConsumer {
   abstract void finishDocument() throws IOException;
   abstract void flush(final SegmentWriteState state) throws IOException;
   abstract void abort();
-  abstract boolean freeRAM();
   abstract void doAfterFlush();
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java Thu Feb  7 20:48:21 2013
@@ -28,15 +28,9 @@ abstract class DocFieldConsumer {
   /** Called when an aborting exception is hit */
   abstract void abort();
 
-  /** Called when DocumentsWriterPerThread is using too much RAM.
-   *  The consumer should free RAM, if possible, returning
-   *  true if any RAM was in fact freed. */
-  abstract boolean freeRAM();
-
   abstract void startDocument() throws IOException;
 
   abstract DocFieldConsumerPerField addField(FieldInfo fi);
 
   abstract void finishDocument() throws IOException;
-
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Thu Feb  7 20:48:21 2013
@@ -25,16 +25,10 @@ import java.util.HashSet;
 import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.FieldInfosWriter;
-import org.apache.lucene.codecs.PerDocConsumer;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
-import org.apache.lucene.index.TypePromoter.TypeCompatibility;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.IOUtils;
-
+import org.apache.lucene.util.Counter;
 
 /**
  * This is a DocConsumer that gathers all fields under the
@@ -47,7 +41,7 @@ import org.apache.lucene.util.IOUtils;
 final class DocFieldProcessor extends DocConsumer {
 
   final DocFieldConsumer consumer;
-  final StoredFieldsConsumer fieldsWriter;
+  final StoredFieldsConsumer storedConsumer;
   final Codec codec;
 
   // Holds all fields seen in current doc
@@ -62,11 +56,14 @@ final class DocFieldProcessor extends Do
   int fieldGen;
   final DocumentsWriterPerThread.DocState docState;
 
-  public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer) {
+  final Counter bytesUsed;
+
+  public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer, StoredFieldsConsumer storedConsumer) {
     this.docState = docWriter.docState;
     this.codec = docWriter.codec;
+    this.bytesUsed = docWriter.bytesUsed;
     this.consumer = consumer;
-    fieldsWriter = new StoredFieldsConsumer(docWriter);
+    this.storedConsumer = storedConsumer;
   }
 
   @Override
@@ -78,15 +75,10 @@ final class DocFieldProcessor extends Do
       childFields.put(f.getFieldInfo().name, f);
     }
 
-    fieldsWriter.flush(state);
-    consumer.flush(childFields, state);
+    assert fields.size() == totalFieldCount;
 
-    for (DocValuesConsumerHolder consumer : docValues.values()) {
-      consumer.docValuesConsumer.finish(state.segmentInfo.getDocCount());
-    }
-    
-    // close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
-    IOUtils.close(perDocConsumer);
+    storedConsumer.flush(state);
+    consumer.flush(childFields, state);
 
     // Important to save after asking consumer to flush so
     // consumer can alter the FieldInfo* if necessary.  EG,
@@ -113,11 +105,9 @@ final class DocFieldProcessor extends Do
         field = next;
       }
     }
-    IOUtils.closeWhileHandlingException(perDocConsumer);
-    // TODO add abort to PerDocConsumer!
     
     try {
-      fieldsWriter.abort();
+      storedConsumer.abort();
     } catch (Throwable t) {
       if (th == null) {
         th = t;
@@ -132,16 +122,6 @@ final class DocFieldProcessor extends Do
       }
     }
     
-    try {
-      if (perDocConsumer != null) {
-        perDocConsumer.abort();  
-      }
-    } catch (Throwable t) {
-      if (th == null) {
-        th = t;
-      }
-    }
-    
     // If any errors occured, throw it.
     if (th != null) {
       if (th instanceof RuntimeException) throw (RuntimeException) th;
@@ -151,11 +131,6 @@ final class DocFieldProcessor extends Do
     }
   }
 
-  @Override
-  public boolean freeRAM() {
-    return consumer.freeRAM();
-  }
-
   public Collection<DocFieldConsumerPerField> fields() {
     Collection<DocFieldConsumerPerField> fields = new HashSet<DocFieldConsumerPerField>();
     for(int i=0;i<fieldHash.length;i++) {
@@ -176,8 +151,6 @@ final class DocFieldProcessor extends Do
     fieldHash = new DocFieldProcessorPerField[2];
     hashMask = 1;
     totalFieldCount = 0;
-    perDocConsumer = null;
-    docValues.clear();
   }
 
   private void rehash() {
@@ -207,7 +180,7 @@ final class DocFieldProcessor extends Do
   public void processDocument(FieldInfos.Builder fieldInfos) throws IOException {
 
     consumer.startDocument();
-    fieldsWriter.startDocument();
+    storedConsumer.startDocument();
 
     fieldCount = 0;
 
@@ -226,38 +199,12 @@ final class DocFieldProcessor extends Do
 
       fp.addField(field);
     }
+
     for (StorableField field: docState.doc.storableFields()) {
       final String fieldName = field.name();
       IndexableFieldType ft = field.fieldType();
-
-      DocFieldProcessorPerField fp = processField(fieldInfos, thisFieldGen, fieldName, ft);
-      if (ft.stored()) {
-        fieldsWriter.addField(field, fp.fieldInfo);
-      }
-      
-      final DocValues.Type dvType = ft.docValueType();
-      if (dvType != null) {
-        DocValuesConsumerHolder docValuesConsumer = docValuesConsumer(dvType,
-            docState, fp.fieldInfo);
-        DocValuesConsumer consumer = docValuesConsumer.docValuesConsumer;
-        if (docValuesConsumer.compatibility == null) {
-          consumer.add(docState.docID, field);
-          docValuesConsumer.compatibility = new TypeCompatibility(dvType,
-              consumer.getValueSize());
-        } else if (docValuesConsumer.compatibility.isCompatible(dvType,
-            TypePromoter.getValueSize(dvType, field.binaryValue()))) {
-          consumer.add(docState.docID, field);
-        } else {
-          docValuesConsumer.compatibility.isCompatible(dvType,
-              TypePromoter.getValueSize(dvType, field.binaryValue()));
-          TypeCompatibility compatibility = docValuesConsumer.compatibility;
-          throw new IllegalArgumentException("Incompatible DocValues type: "
-              + dvType.name() + " size: "
-              + TypePromoter.getValueSize(dvType, field.binaryValue())
-              + " expected: " + " type: " + compatibility.getBaseType()
-              + " size: " + compatibility.getBaseSize());
-        }
-      }
+      FieldInfo fieldInfo = fieldInfos.addOrUpdate(fieldName, ft);
+      storedConsumer.addField(docState.docID, field, fieldInfo);
     }
 
     // If we are writing vectors then we must visit
@@ -280,6 +227,7 @@ final class DocFieldProcessor extends Do
 
   private DocFieldProcessorPerField processField(FieldInfos.Builder fieldInfos,
       final int thisFieldGen, final String fieldName, IndexableFieldType ft) {
+
     // Make sure we have a PerField allocated
     final int hashPos = fieldName.hashCode() & hashMask;
     DocFieldProcessorPerField fp = fieldHash[hashPos];
@@ -305,7 +253,7 @@ final class DocFieldProcessor extends Do
         rehash();
       }
     } else {
-      fieldInfos.addOrUpdate(fp.fieldInfo.name, ft);
+      fp.fieldInfo.update(ft);
     }
 
     if (thisFieldGen != fp.lastGen) {
@@ -336,54 +284,9 @@ final class DocFieldProcessor extends Do
   @Override
   void finishDocument() throws IOException {
     try {
-      fieldsWriter.finishDocument();
+      storedConsumer.finishDocument();
     } finally {
       consumer.finishDocument();
     }
   }
-
-  private static class DocValuesConsumerHolder {
-    // Only used to enforce that same DV field name is never
-    // added more than once per doc:
-    int docID;
-    final DocValuesConsumer docValuesConsumer;
-    TypeCompatibility compatibility;
-
-    public DocValuesConsumerHolder(DocValuesConsumer docValuesConsumer) {
-      this.docValuesConsumer = docValuesConsumer;
-    }
-  }
-
-  final private Map<String, DocValuesConsumerHolder> docValues = new HashMap<String, DocValuesConsumerHolder>();
-  private PerDocConsumer perDocConsumer;
-
-  DocValuesConsumerHolder docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo) 
-      throws IOException {
-    DocValuesConsumerHolder docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
-    if (docValuesConsumerAndDocID != null) {
-      if (docState.docID == docValuesConsumerAndDocID.docID) {
-        throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
-      }
-      assert docValuesConsumerAndDocID.docID < docState.docID;
-      docValuesConsumerAndDocID.docID = docState.docID;
-      return docValuesConsumerAndDocID;
-    }
-
-    if (perDocConsumer == null) {
-      PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState("");
-      perDocConsumer = docState.docWriter.codec.docValuesFormat().docsConsumer(perDocWriteState);
-      if (perDocConsumer == null) {
-        throw new IllegalStateException("codec=" +  docState.docWriter.codec + " does not support docValues: from docValuesFormat().docsConsumer(...) returned null; field=" + fieldInfo.name);
-      }
-    }
-    DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
-    assert fieldInfo.getDocValuesType() == null || fieldInfo.getDocValuesType() == valueType;
-    fieldInfo.setDocValuesType(valueType);
-
-    docValuesConsumerAndDocID = new DocValuesConsumerHolder(docValuesConsumer);
-    docValuesConsumerAndDocID.docID = docState.docID;
-    docValues.put(fieldInfo.name, docValuesConsumerAndDocID);
-    return docValuesConsumerAndDocID;
-  }
-  
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java Thu Feb  7 20:48:21 2013
@@ -17,7 +17,12 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.RamUsageEstimator;
 
 /**
@@ -28,16 +33,19 @@ final class DocFieldProcessorPerField {
 
   final DocFieldConsumerPerField consumer;
   final FieldInfo fieldInfo;
+  private final Counter bytesUsed;
 
   DocFieldProcessorPerField next;
   int lastGen = -1;
 
   int fieldCount;
   IndexableField[] fields = new IndexableField[1];
+  private final Map<FieldInfo,String> dvFields = new HashMap<FieldInfo,String>();
 
   public DocFieldProcessorPerField(final DocFieldProcessor docFieldProcessor, final FieldInfo fieldInfo) {
     this.consumer = docFieldProcessor.consumer.addField(fieldInfo);
     this.fieldInfo = fieldInfo;
+    this.bytesUsed = docFieldProcessor.bytesUsed;
   }
 
   public void addField(IndexableField field) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocInverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocInverter.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocInverter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocInverter.java Thu Feb  7 20:48:21 2013
@@ -78,13 +78,7 @@ final class DocInverter extends DocField
   }
 
   @Override
-  public boolean freeRAM() {
-    return consumer.freeRAM();
-  }
-
-  @Override
   public DocFieldConsumerPerField addField(FieldInfo fi) {
     return new DocInverterPerField(this, fi);
   }
-
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Thu Feb  7 20:48:21 2013
@@ -63,18 +63,21 @@ class DocumentsWriterPerThread {
       This is the current indexing chain:
 
       DocConsumer / DocConsumerPerThread
-        --> code: DocFieldProcessor / DocFieldProcessorPerThread
-          --> DocFieldConsumer / DocFieldConsumerPerThread / DocFieldConsumerPerField
-            --> code: DocFieldConsumers / DocFieldConsumersPerThread / DocFieldConsumersPerField
-              --> code: DocInverter / DocInverterPerThread / DocInverterPerField
-                --> InvertedDocConsumer / InvertedDocConsumerPerThread / InvertedDocConsumerPerField
-                  --> code: TermsHash / TermsHashPerThread / TermsHashPerField
-                    --> TermsHashConsumer / TermsHashConsumerPerThread / TermsHashConsumerPerField
-                      --> code: FreqProxTermsWriter / FreqProxTermsWriterPerThread / FreqProxTermsWriterPerField
-                      --> code: TermVectorsTermsWriter / TermVectorsTermsWriterPerThread / TermVectorsTermsWriterPerField
-                --> InvertedDocEndConsumer / InvertedDocConsumerPerThread / InvertedDocConsumerPerField
-                  --> code: NormsWriter / NormsWriterPerThread / NormsWriterPerField
-              --> code: StoredFieldsWriter / StoredFieldsWriterPerThread / StoredFieldsWriterPerField
+        --> code: DocFieldProcessor
+          --> DocFieldConsumer / DocFieldConsumerPerField
+            --> code: DocFieldConsumers / DocFieldConsumersPerField
+              --> code: DocInverter / DocInverterPerField
+                --> InvertedDocConsumer / InvertedDocConsumerPerField
+                  --> code: TermsHash / TermsHashPerField
+                    --> TermsHashConsumer / TermsHashConsumerPerField
+                      --> code: FreqProxTermsWriter / FreqProxTermsWriterPerField
+                      --> code: TermVectorsTermsWriter / TermVectorsTermsWriterPerField
+                --> InvertedDocEndConsumer / InvertedDocConsumerPerField
+                  --> code: NormsConsumer / NormsConsumerPerField
+          --> StoredFieldsConsumer
+            --> TwoStoredFieldConsumers
+              -> code: StoredFieldsProcessor
+              -> code: DocValuesProcessor
     */
 
     // Build up indexing chain:
@@ -82,11 +85,14 @@ class DocumentsWriterPerThread {
       final TermsHashConsumer termVectorsWriter = new TermVectorsConsumer(documentsWriterPerThread);
       final TermsHashConsumer freqProxWriter = new FreqProxTermsWriter();
 
-      final InvertedDocConsumer  termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
-                                                           new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null));
-      final NormsConsumer normsWriter = new NormsConsumer(documentsWriterPerThread);
+      final InvertedDocConsumer termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
+                                                          new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null));
+      final NormsConsumer normsWriter = new NormsConsumer();
       final DocInverter docInverter = new DocInverter(documentsWriterPerThread.docState, termsHash, normsWriter);
-      return new DocFieldProcessor(documentsWriterPerThread, docInverter);
+      final StoredFieldsConsumer storedFields = new TwoStoredFieldsConsumers(
+                                                      new StoredFieldsProcessor(documentsWriterPerThread),
+                                                      new DocValuesProcessor(documentsWriterPerThread.bytesUsed));
+      return new DocFieldProcessor(documentsWriterPerThread, docInverter, storedFields);
     }
   };
 
@@ -651,10 +657,6 @@ class DocumentsWriterPerThread {
     }
     
   }
-  PerDocWriteState newPerDocWriteState(String segmentSuffix) {
-    assert segmentInfo != null;
-    return new PerDocWriteState(infoStream, directory, segmentInfo, bytesUsed, segmentSuffix, IOContext.DEFAULT);
-  }
   
   @Override
   public String toString() {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Thu Feb  7 20:48:21 2013
@@ -20,8 +20,6 @@ package org.apache.lucene.index;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.lucene.index.DocValues.Type;
-
 /**
  *  Access to the Field Info file that describes document fields and whether or
  *  not they are indexed. Each segment has a separate Field Info file. Objects
@@ -37,12 +35,12 @@ public final class FieldInfo {
   public final int number;
 
   private boolean indexed;
-  private DocValues.Type docValueType;
+  private DocValuesType docValueType;
 
   // True if any document indexed term vectors
   private boolean storeTermVector;
 
-  private DocValues.Type normType;
+  private DocValuesType normType;
   private boolean omitNorms; // omit norms associated with indexed fields  
   private IndexOptions indexOptions;
   private boolean storePayloads; // whether this field stores payloads together with term positions
@@ -82,6 +80,29 @@ public final class FieldInfo {
      */
     DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
   };
+  
+  /**
+   * DocValues types.
+   * Note that DocValues is strongly typed, so a field cannot have different types
+   * across different documents.
+   */
+  public static enum DocValuesType {
+    /** 
+     * A per-document Number
+     */
+    NUMERIC,
+    /**
+     * A per-document byte[].
+     */
+    BINARY,
+    /** 
+     * A pre-sorted byte[]. Fields with this type only store distinct byte values 
+     * and store an additional offset pointer per document to dereference the shared 
+     * byte[]. The stored byte[] is presorted and allows access via document id, 
+     * ordinal and by-value.
+     */
+    SORTED
+  };
 
   /**
    * Sole Constructor.
@@ -89,7 +110,7 @@ public final class FieldInfo {
    * @lucene.experimental
    */
   public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector, 
-            boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normsType, Map<String,String> attributes) {
+            boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType, Map<String,String> attributes) {
     this.name = name;
     this.indexed = indexed;
     this.number = number;
@@ -130,9 +151,13 @@ public final class FieldInfo {
     return true;
   }
 
+  void update(IndexableFieldType ft) {
+    update(ft.indexed(), false, ft.omitNorms(), false, ft.indexOptions());
+  }
+
   // should only be called by FieldInfos#addOrUpdate
   void update(boolean indexed, boolean storeTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions) {
-
+    //System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" + omitNorms + " this.omitNorms=" + this.omitNorms);
     if (this.indexed != indexed) {
       this.indexed = true;                      // once indexed, always index
     }
@@ -163,7 +188,10 @@ public final class FieldInfo {
     assert checkConsistency();
   }
 
-  void setDocValuesType(DocValues.Type type) {
+  void setDocValuesType(DocValuesType type) {
+    if (docValueType != null && docValueType != type) {
+      throw new IllegalArgumentException("cannot change DocValues type from " + docValueType + " to " + type + " for field \"" + name + "\"");
+    }
     docValueType = type;
     assert checkConsistency();
   }
@@ -181,16 +209,16 @@ public final class FieldInfo {
   }
 
   /**
-   * Returns {@link DocValues.Type} of the docValues. this may be null if the field has no docvalues.
+   * Returns {@link DocValuesType} of the docValues. this may be null if the field has no docvalues.
    */
-  public DocValues.Type getDocValuesType() {
+  public DocValuesType getDocValuesType() {
     return docValueType;
   }
   
   /**
-   * Returns {@link DocValues.Type} of the norm. this may be null if the field has no norms.
+   * Returns {@link DocValuesType} of the norm. this may be null if the field has no norms.
    */
-  public DocValues.Type getNormType() {
+  public DocValuesType getNormType() {
     return normType;
   }
 
@@ -206,7 +234,10 @@ public final class FieldInfo {
     assert checkConsistency();
   }
 
-  void setNormValueType(Type type) {
+  void setNormValueType(DocValuesType type) {
+    if (normType != null && normType != type) {
+      throw new IllegalArgumentException("cannot change Norm type from " + normType + " to " + type + " for field \"" + name + "\"");
+    }
     normType = type;
     assert checkConsistency();
   }