You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/01/11 17:51:57 UTC

svn commit: r1230141 [1/2] - in /lucene/dev/branches/lucene3453: lucene/contrib/misc/src/java/org/apache/lucene/document/ lucene/src/java/org/apache/lucene/codecs/ lucene/src/java/org/apache/lucene/codecs/lucene40/ lucene/src/java/org/apache/lucene/cod...

Author: mikemccand
Date: Wed Jan 11 16:51:55 2012
New Revision: 1230141

URL: http://svn.apache.org/viewvc?rev=1230141&view=rev
Log:
LUCENE-3453: initial patch

Modified:
    lucene/dev/branches/lucene3453/lucene/contrib/misc/src/java/org/apache/lucene/document/LazyDocument.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/DocValuesField.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/Field.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/FieldType.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/StringField.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocValue.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableField.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableFieldType.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumer.java
    lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java
    lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexNormsConsumer.java
    lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/index/RandomIndexWriter.java
    lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/codecs/lucene40/TestDocValues.java
    lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
    lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
    lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDuelingCodecs.java
    lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestIndexableField.java
    lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestTypePromotion.java
    lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java
    lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/search/TestSort.java
    lucene/dev/branches/lucene3453/modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
    lucene/dev/branches/lucene3453/modules/grouping/src/test/org/apache/lucene/search/grouping/AllGroupsCollectorTest.java
    lucene/dev/branches/lucene3453/modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
    lucene/dev/branches/lucene3453/solr/core/src/java/org/apache/solr/schema/TrieField.java

Modified: lucene/dev/branches/lucene3453/lucene/contrib/misc/src/java/org/apache/lucene/document/LazyDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/contrib/misc/src/java/org/apache/lucene/document/LazyDocument.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/contrib/misc/src/java/org/apache/lucene/document/LazyDocument.java (original)
+++ lucene/dev/branches/lucene3453/lucene/contrib/misc/src/java/org/apache/lucene/document/LazyDocument.java Wed Jan 11 16:51:55 2012
@@ -28,8 +28,6 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.IndexableFieldType;
-import org.apache.lucene.index.DocValue;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.util.BytesRef;
 
 /** Defers actually loading a field's value until you ask
@@ -121,15 +119,6 @@ public class LazyDocument {
     }
 
     @Override
-    public boolean numeric() {
-      if (num == 0) {
-        return getDocument().getField(name).numeric();
-      } else {
-        return getDocument().getFields(name)[num].numeric();
-      }
-    }
-
-    @Override
     public DataType numericDataType() {
       if (num == 0) {
         return getDocument().getField(name).numericDataType();
@@ -157,24 +146,6 @@ public class LazyDocument {
     }
 
     @Override
-    public DocValue docValue() {
-      if (num == 0) {
-        return getDocument().getField(name).docValue();
-      } else {
-        return getDocument().getFields(name)[num].docValue();
-      }
-    }
-
-    @Override
-    public DocValues.Type docValueType() {
-      if (num == 0) {
-        return getDocument().getField(name).docValueType();
-      } else {
-        return getDocument().getFields(name)[num].docValueType();
-      }
-    }
-
-    @Override
     public TokenStream tokenStream(Analyzer analyzer) throws IOException {
       if (num == 0) {
         return getDocument().getField(name).tokenStream(analyzer);

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Wed Jan 11 16:51:55 2012
@@ -19,11 +19,11 @@ package org.apache.lucene.codecs;
 import java.io.IOException;
 
 import org.apache.lucene.codecs.lucene40.values.Writer;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValues.Source;
+import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.DocValue;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
@@ -31,7 +31,7 @@ import org.apache.lucene.util.BytesRef;
  * Abstract API that consumes {@link DocValue}s.
  * {@link DocValuesConsumer} are always associated with a specific field and
  * segments. Concrete implementations of this API write the given
- * {@link DocValue} into a implementation specific format depending on
+ * {@link IndexableField} into a implementation specific format depending on
  * the fields meta-data.
  * 
  * @lucene.experimental
@@ -42,28 +42,27 @@ public abstract class DocValuesConsumer 
   protected final BytesRef spare = new BytesRef();
 
   /**
-   * Adds the given {@link DocValue} instance to this
+   * Adds the given {@link IndexableField} instance to this
    * {@link DocValuesConsumer}
    * 
    * @param docID
    *          the document ID to add the value for. The docID must always
    *          increase or be <tt>0</tt> if it is the first call to this method.
-   * @param docValue
+   * @param value
    *          the value to add
    * @throws IOException
    *           if an {@link IOException} occurs
    */
-  public abstract void add(int docID, DocValue docValue)
+  public abstract void add(int docID, IndexableField value)
       throws IOException;
 
   /**
-   * Called when the consumer of this API is doc with adding
-   * {@link DocValue} to this {@link DocValuesConsumer}
+   * Called when the consumer of this API is done adding values.
    * 
    * @param docCount
    *          the total number of documents in this {@link DocValuesConsumer}.
    *          Must be greater than or equal the last given docID to
-   *          {@link #add(int, DocValue)}.
+   *          {@link #add(int, IndexableField)}.
    * @throws IOException
    */
   public abstract void finish(int docCount) throws IOException;

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java Wed Jan 11 16:51:55 2012
@@ -21,13 +21,14 @@ import java.io.IOException;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.NumericField;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.MergePolicy.MergeAbortedException;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentReader;
-import org.apache.lucene.index.MergePolicy.MergeAbortedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -127,7 +128,7 @@ public final class Lucene40StoredFieldsW
         IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
   }
 
-  public final void writeField(FieldInfo info, IndexableField field) throws IOException {
+  public void writeField(FieldInfo info, IndexableField field) throws IOException {
     fieldsStream.writeVInt(info.number);
     int bits = 0;
     final BytesRef bytes;
@@ -136,8 +137,9 @@ public final class Lucene40StoredFieldsW
     // this way we don't bake into indexer all these
     // specific encodings for different fields?  and apps
     // can customize...
-    if (field.numeric()) {
-      switch (field.numericDataType()) {
+    final NumericField.DataType numericType = field.numericDataType();
+    if (numericType != null) {
+      switch (numericType) {
         case INT:
           bits |= FIELD_IS_NUMERIC_INT; break;
         case LONG:
@@ -193,7 +195,7 @@ public final class Lucene40StoredFieldsW
    *  document.  The stream IndexInput is the
    *  fieldsStream from which we should bulk-copy all
    *  bytes. */
-  public final void addRawDocuments(IndexInput stream, int[] lengths, int numDocs) throws IOException {
+  public void addRawDocuments(IndexInput stream, int[] lengths, int numDocs) throws IOException {
     long position = fieldsStream.getFilePointer();
     long start = position;
     for(int i=0;i<numDocs;i++) {

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java Wed Jan 11 16:51:55 2012
@@ -23,12 +23,12 @@ import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.DocValue;
 import org.apache.lucene.index.DocValues.SortedSource;
 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.IndexFileNames;
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -308,9 +308,9 @@ public final class Bytes {
     }
 
     @Override
-    public void add(int docID, DocValue docValue) throws IOException {
-      final BytesRef ref;
-      if ((ref = docValue.getBytes()) != null) {
+    public void add(int docID, IndexableField docValue) throws IOException {
+      final BytesRef ref = docValue.binaryValue();
+      if (ref != null) {
         add(docID, ref);
       }
     }

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java Wed Jan 11 16:51:55 2012
@@ -69,12 +69,11 @@ class FixedStraightBytesImpl {
 
       if (size == -1) {
         if (bytes.length > BYTE_BLOCK_SIZE) {
-          throw new IllegalArgumentException("bytes arrays > " + Short.MAX_VALUE + " are not supported");
+          throw new IllegalArgumentException("bytes arrays > " + BYTE_BLOCK_SIZE + " are not supported");
         }
         size = bytes.length;
       } else if (bytes.length != size) {
-        throw new IllegalArgumentException("expected bytes size=" + size
-            + " but got " + bytes.length);
+        throw new IllegalArgumentException("byte[] length changed for BYTES_FIXED_STRAIGHT type (before=" + size + " now=" + bytes.length);
       }
       if (lastDocID+1 < docID) {
         advancePool(docID);
@@ -204,8 +203,6 @@ class FixedStraightBytesImpl {
       currentMergeSource.getBytes(sourceDoc, bytesRef);
     }
 
-
-
     // Fills up to but not including this docID
     private void fill(IndexOutput datOut, int docID) throws IOException {
       assert size >= 0;

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java Wed Jan 11 16:51:55 2012
@@ -19,10 +19,10 @@ package org.apache.lucene.codecs.lucene4
 import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValue;
 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;
@@ -86,8 +86,11 @@ public class Floats {
     }
     
     @Override
-    public void add(int docID, DocValue docValue) throws IOException {
-      add(docID, docValue.getFloat());
+    public void add(int docID, IndexableField docValue) throws IOException {
+      Number number = docValue.numericValue();
+      if (number != null) {
+        add(docID, number.doubleValue());
+      }
     }
     
     @Override

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java Wed Jan 11 16:51:55 2012
@@ -20,9 +20,9 @@ package org.apache.lucene.codecs.lucene4
 import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValues.Type;
-import org.apache.lucene.index.DocValue;
+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;
@@ -110,8 +110,11 @@ public final class Ints {
     }
 
     @Override
-    public void add(int docID, DocValue docValue) throws IOException {
-      add(docID, docValue.getInt());
+    public void add(int docID, IndexableField docValue) throws IOException {
+      final Number number = docValue.numericValue();
+      if (number != null) {
+        add(docID, number.longValue());
+      }
     }
     
     @Override

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java Wed Jan 11 16:51:55 2012
@@ -20,11 +20,11 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.lucene40.values.DocValuesArray.LongValues;
 import org.apache.lucene.codecs.lucene40.values.FixedStraightBytesImpl.FixedBytesWriterBase;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.DocValue;
 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.IndexFileNames;
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -151,8 +151,11 @@ class PackedIntValues {
     }
 
     @Override
-    public void add(int docID, DocValue docValue) throws IOException {
-      add(docID, docValue.getInt());
+    public void add(int docID, IndexableField docValue) throws IOException {
+      final Number number = docValue.numericValue();
+      if (number != null) {
+        add(docID, number.longValue());
+      }
     }
   }
 

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java Wed Jan 11 16:51:55 2012
@@ -23,13 +23,13 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.PerDocConsumer;
-import org.apache.lucene.index.DocValue;
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -130,8 +130,8 @@ public class SimpleTextNormsConsumer ext
     }
 
     @Override
-    public void add(int docID, DocValue docValue) throws IOException {
-      add(docID, docValue.getBytes());
+    public void add(int docID, IndexableField docValue) throws IOException {
+      add(docID, docValue.binaryValue());
     }
     
     protected void add(int docID, BytesRef value) throws IOException {

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java Wed Jan 11 16:51:55 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.simplet
 import java.io.IOException;
 
 import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.document.NumericField;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexableField;
@@ -98,8 +99,10 @@ public class SimpleTextStoredFieldsWrite
     newLine();
     
     write(TYPE);
-    if (field.numeric()) {
-      switch (field.numericDataType()) {
+    final NumericField.DataType numericType = field.numericDataType();
+
+    if (numericType != null) {
+      switch (numericType) {
         case INT:
           write(TYPE_INT);
           newLine();

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/DocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/DocValuesField.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/DocValuesField.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/DocValuesField.java Wed Jan 11 16:51:55 2012
@@ -20,7 +20,6 @@ import java.io.Reader;
 import java.util.Comparator;
 
 import org.apache.lucene.index.IndexableFieldType;
-import org.apache.lucene.index.DocValue;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValues.Type; // javadocs
 import org.apache.lucene.util.BytesRef;
@@ -69,167 +68,90 @@ import org.apache.lucene.util.BytesRef;
  * </pre>
  * 
  * */
-public class DocValuesField extends Field implements DocValue {
+
+// nocommit -- how to sugar this...?
+
+public class DocValuesField extends Field {
 
   protected BytesRef bytes;
-  protected double doubleValue;
-  protected long longValue;
-  protected DocValues.Type type;
+  protected Number numberValue;
   protected Comparator<BytesRef> bytesComparator;
 
+  // nocommit sugar ctors taking byte, short, int, etc.?
+
   /**
    * Creates a new {@link DocValuesField} with the given name.
    */
-  public DocValuesField(String name) {
-    this(name, new FieldType());
+  public DocValuesField(String name, DocValues.Type docValueType) {
+    super(name, new FieldType());
+    if (docValueType == null) {
+      throw new NullPointerException("docValueType cannot be null");
+    }
+    FieldType ft = (FieldType) type;
+    ft.setDocValueType(docValueType);
+    ft.freeze();
   }
 
   public DocValuesField(String name, IndexableFieldType type) {
-    this(name, type, null);
-  }
-
-  public DocValuesField(String name, IndexableFieldType type, String value) {
     super(name, type);
-    fieldsData = value;
-  }
-
-  @Override
-  public DocValue docValue() {
-    return this;
+    if (type.docValueType() == null) {
+      throw new NullPointerException("docValueType cannot be null");
+    }
   }
 
   /**
-   * Sets the given <code>long</code> value and sets the field's {@link Type} to
-   * {@link Type#VAR_INTS} unless already set. If you want to change the
-   * default type use {@link #setDocValuesType(DocValues.Type)}.
+   * Sets the given <code>long</code> value.
    */
   public void setInt(long value) {
-    setInt(value, false);
-  }
-  
-  /**
-   * Sets the given <code>long</code> value as a 64 bit signed integer.
-   * 
-   * @param value
-   *          the value to set
-   * @param fixed
-   *          if <code>true</code> {@link Type#FIXED_INTS_64} is used
-   *          otherwise {@link Type#VAR_INTS}
-   */
-  public void setInt(long value, boolean fixed) {
-    if (type == null) {
-      type = fixed ? DocValues.Type.FIXED_INTS_64 : DocValues.Type.VAR_INTS;
-    }
-    longValue = value;
+    // nocommit assert type matches
+    numberValue = value;
   }
 
   /**
-   * Sets the given <code>int</code> value and sets the field's {@link Type} to
-   * {@link Type#VAR_INTS} unless already set. If you want to change the
-   * default type use {@link #setDocValuesType(DocValues.Type)}.
+   * Sets the given <code>int</code> value.
    */
   public void setInt(int value) {
-    setInt(value, false);
+    // nocommit assert type matches
+    numberValue = value;
   }
 
   /**
-   * Sets the given <code>int</code> value as a 32 bit signed integer.
-   * 
-   * @param value
-   *          the value to set
-   * @param fixed
-   *          if <code>true</code> {@link Type#FIXED_INTS_32} is used
-   *          otherwise {@link Type#VAR_INTS}
-   */
-  public void setInt(int value, boolean fixed) {
-    if (type == null) {
-      type = fixed ? DocValues.Type.FIXED_INTS_32 : DocValues.Type.VAR_INTS;
-    }
-    longValue = value;
-  }
-
-  /**
-   * Sets the given <code>short</code> value and sets the field's {@link Type} to
-   * {@link Type#VAR_INTS} unless already set. If you want to change the
-   * default type use {@link #setDocValuesType(DocValues.Type)}.
+   * Sets the given <code>short</code> value.
    */
   public void setInt(short value) {
-    setInt(value, false);
+    // nocommit assert type matches
+    numberValue = value;
   }
 
   /**
-   * Sets the given <code>short</code> value as a 16 bit signed integer.
-   * 
-   * @param value
-   *          the value to set
-   * @param fixed
-   *          if <code>true</code> {@link Type#FIXED_INTS_16} is used
-   *          otherwise {@link Type#VAR_INTS}
-   */
-  public void setInt(short value, boolean fixed) {
-    if (type == null) {
-      type = fixed ? DocValues.Type.FIXED_INTS_16 : DocValues.Type.VAR_INTS;
-    }
-    longValue = value;
-  }
-
-  /**
-   * Sets the given <code>byte</code> value and sets the field's {@link Type} to
-   * {@link Type#VAR_INTS} unless already set. If you want to change the
-   * default type use {@link #setDocValuesType(DocValues.Type)}.
+   * Sets the given <code>byte</code> value.
    */
   public void setInt(byte value) {
-    setInt(value, false);
-  }
-
-  /**
-   * Sets the given <code>byte</code> value as a 8 bit signed integer.
-   * 
-   * @param value
-   *          the value to set
-   * @param fixed
-   *          if <code>true</code> {@link Type#FIXED_INTS_8} is used
-   *          otherwise {@link Type#VAR_INTS}
-   */
-  public void setInt(byte value, boolean fixed) {
-    if (type == null) {
-      type = fixed ? DocValues.Type.FIXED_INTS_8 : DocValues.Type.VAR_INTS;
-    }
-    longValue = value;
+    // nocommit assert type matches
+    numberValue = value;
   }
 
   /**
-   * Sets the given <code>float</code> value and sets the field's {@link Type}
-   * to {@link Type#FLOAT_32} unless already set. If you want to
-   * change the type use {@link #setDocValuesType(DocValues.Type)}.
+   * Sets the given <code>float</code> value.
    */
   public void setFloat(float value) {
-    if (type == null) {
-      type = DocValues.Type.FLOAT_32;
-    }
-    doubleValue = value;
+    // nocommit assert type matches
+    numberValue = value;
   }
 
   /**
-   * Sets the given <code>double</code> value and sets the field's {@link Type}
-   * to {@link Type#FLOAT_64} unless already set. If you want to
-   * change the default type use {@link #setDocValuesType(DocValues.Type)}.
+   * Sets the given <code>double</code> value.
    */
   public void setFloat(double value) {
-    if (type == null) {
-      type = DocValues.Type.FLOAT_64;
-    }
-    doubleValue = value;
+    // nocommit assert type matches
+    numberValue = value;
   }
 
   /**
-   * Sets the given {@link BytesRef} value and the field's {@link Type}. The
-   * comparator for this field is set to <code>null</code>. If a
-   * <code>null</code> comparator is set the default comparator for the given
-   * {@link Type} is used.
+   * Sets the given {@link BytesRef} value.
    */
-  public void setBytes(BytesRef value, DocValues.Type type) {
-    setBytes(value, type, null);
+  public void setBytes(BytesRef value) {
+    bytes = value;
   }
 
   /**
@@ -240,6 +162,8 @@ public class DocValuesField extends Fiel
    * @throws IllegalArgumentException
    *           if the value or the type are null
    */
+  // nocommit what to do w/ comparator...
+  /*
   public void setBytes(BytesRef value, DocValues.Type type, Comparator<BytesRef> comp) {
     if (value == null) {
       throw new IllegalArgumentException("value must not be null");
@@ -252,69 +176,49 @@ public class DocValuesField extends Fiel
     }
     bytesComparator = comp;
   }
+  */
 
-  /**
-   * Returns the set {@link BytesRef} or <code>null</code> if not set.
-   */
-  public BytesRef getBytes() {
+  @Override
+  public BytesRef binaryValue() {
     return bytes;
   }
 
   /**
    * Returns the set {@link BytesRef} comparator or <code>null</code> if not set
    */
+  /*
   public Comparator<BytesRef> bytesComparator() {
     return bytesComparator;
   }
+  */
 
-  /**
-   * Returns the set floating point value or <code>0.0d</code> if not set.
-   */
-  public double getFloat() {
-    return doubleValue;
-  }
-
-  /**
-   * Returns the set <code>long</code> value of <code>0</code> if not set.
-   */
-  public long getInt() {
-    return longValue;
+  @Override
+  public Number numericValue() {
+    return numberValue;
   }
 
   /**
    * Sets the {@link BytesRef} comparator for this field. If the field has a
    * numeric {@link Type} the comparator will be ignored.
    */
+  /*
   public void setBytesComparator(Comparator<BytesRef> comp) {
     this.bytesComparator = comp;
   }
-
-  /**
-   * Sets the {@link Type} for this field.
-   */
-  public void setDocValuesType(DocValues.Type type) {
-    if (type == null) {
-      throw new IllegalArgumentException("Type must not be null");
-    }
-    this.type = type;
-  }
+  */
 
   /**
    * Returns always <code>null</code>
    */
+  @Override
   public Reader readerValue() {
     return null;
   }
 
   @Override
-  public DocValues.Type docValueType() {
-    return type;
-  }
-
-  @Override
   public String toString() {
     final String value;
-    switch (type) {
+    switch (type.docValueType()) {
     case BYTES_FIXED_DEREF:
     case BYTES_FIXED_STRAIGHT:
     case BYTES_VAR_DEREF:
@@ -325,70 +229,29 @@ public class DocValuesField extends Fiel
       value = "bytes: " + bytes.toString();
       break;
     case FIXED_INTS_16:
-      value = "int16: " + longValue;
+      value = "int16: " + numberValue;
       break;
     case FIXED_INTS_32:
-      value = "int32: " + longValue;
+      value = "int32: " + numberValue;
       break;
     case FIXED_INTS_64:
-      value = "int64: " + longValue;
+      value = "int64: " + numberValue;
       break;
     case FIXED_INTS_8:
-      value = "int8: " + longValue;
+      value = "int8: " + numberValue;
       break;
     case VAR_INTS:
-      value = "vint: " + longValue;
+      value = "vint: " + numberValue;
       break;
     case FLOAT_32:
-      value = "float32: " + doubleValue;
+      value = "float32: " + numberValue;
       break;
     case FLOAT_64:
-      value = "float64: " + doubleValue;
+      value = "float64: " + numberValue;
       break;
     default:
       throw new IllegalArgumentException("unknown type: " + type);
     }
     return "<" + name() + ": DocValuesField " + value + ">";
   }
-
-  /**
-   * Returns an DocValuesField holding the value from
-   * the provided string field, as the specified type.  The
-   * incoming field must have a string value.  The name, {@link
-   * FieldType} and string value are carried over from the
-   * incoming Field.
-   */
-  public static DocValuesField build(Field field, DocValues.Type type) {
-    if (field instanceof DocValuesField) {
-      return (DocValuesField) field;
-    }
-    final DocValuesField valField = new DocValuesField(field.name(), field.fieldType(), field.stringValue());
-    switch (type) {
-    case BYTES_FIXED_DEREF:
-    case BYTES_FIXED_STRAIGHT:
-    case BYTES_VAR_DEREF:
-    case BYTES_VAR_STRAIGHT:
-    case BYTES_FIXED_SORTED:
-    case BYTES_VAR_SORTED:
-      BytesRef ref = field.isBinary() ? field.binaryValue() : new BytesRef(field.stringValue());
-      valField.setBytes(ref, type);
-      break;
-    case FIXED_INTS_16:
-    case FIXED_INTS_32:
-    case FIXED_INTS_64:
-    case FIXED_INTS_8:
-    case VAR_INTS:
-      valField.setInt(Long.parseLong(field.stringValue()));
-      break;
-    case FLOAT_32:
-      valField.setFloat(Float.parseFloat(field.stringValue()));
-      break;
-    case FLOAT_64:
-      valField.setFloat(Double.parseDouble(field.stringValue()));
-      break;
-    default:
-      throw new IllegalArgumentException("unknown type: " + type);
-    }
-    return valField;
-  }
 }

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/Field.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/Field.java Wed Jan 11 16:51:55 2012
@@ -25,37 +25,38 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexableFieldType;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.DocValue;
 import org.apache.lucene.util.BytesRef;
 
 /**
- * A field is a section of a Document. Each field has two parts, a name and a
- * value. Values may be free text, provided as a String or as a Reader, or they
- * may be atomic keywords, which are not further processed. Such keywords may be
- * used to represent dates, urls, etc. Fields are optionally stored in the
+ * A field is a section of a Document. Each field has three
+ * parts: name, type andvalue. Values may be text
+ * (String, Reader or pre-analyzed TokenStream), binary
+ * (byte[]), or numeric (a Number).  Fields are optionally stored in the
  * index, so that they may be returned with hits on the document.
  * <p/>
- * Note, Field instances are instantiated with a {@link IndexableFieldType}.  Making changes
- * to the state of the FieldType will impact any Field it is used in, therefore
- * it is strongly recommended that no changes are made after Field instantiation.
+ * NOTE: the field type is an {@link IndexableFieldType}.  Making changes
+ * to the state of the IndexableFieldType will impact any
+ * Field it is used in.  It is strongly recommended that no
+ * changes be made after Field instantiation.
  */
 public class Field implements IndexableField {
   
-  protected IndexableFieldType type;
-  protected String name = "body";
+  protected final IndexableFieldType type;
+  protected final String name;
   // the data object for all different kind of field values
   protected Object fieldsData;
+
+  // nocommit why not stuffed into fieldsData...?  hmm are you
+  // allowed to have field value *and* tokenStream?... messy
   // pre-analyzed tokenStream for indexed fields
   protected TokenStream tokenStream;
   // length/offset for all primitive types
-  protected DocValue docValue;
-  
+
   protected float boost = 1.0f;
 
-  public Field(String name, IndexableFieldType type) {
+  protected Field(String name, IndexableFieldType type) {
     this.name = name;
     this.type = type;
   }
@@ -75,7 +76,7 @@ public class Field implements IndexableF
     this.fieldsData = reader;
     this.type = type;
   }
-  
+
   public Field(String name, TokenStream tokenStream, IndexableFieldType type) {
     if (name == null) {
       throw new NullPointerException("name cannot be null");
@@ -83,6 +84,12 @@ public class Field implements IndexableF
     if (tokenStream == null) {
       throw new NullPointerException("tokenStream cannot be null");
     }
+    if (!type.indexed()) {
+      throw new IllegalArgumentException("TokenStream fields must be indexed");
+    }
+    if (type.stored()) {
+      throw new IllegalArgumentException("TokenStream fields cannot be stored");
+    }
     if (type.indexed() && !type.tokenized()) {
       throw new IllegalArgumentException("Non-tokenized fields must use String values");
     }
@@ -101,6 +108,8 @@ public class Field implements IndexableF
     this(name, new BytesRef(value, offset, length), type);
   }
 
+  // nocommit numerics ctors too
+
   public Field(String name, BytesRef bytes, IndexableFieldType type) {
     if (type.indexed() && !type.tokenized()) {
       throw new IllegalArgumentException("Non-tokenized fields must use String values");
@@ -110,6 +119,8 @@ public class Field implements IndexableF
     this.type = type;
     this.name = name;
   }
+
+  // nocommit test case for LUCENE-3616 (and other invalid combos)
   
   public Field(String name, String value, IndexableFieldType type) {
     if (name == null) {
@@ -119,10 +130,12 @@ public class Field implements IndexableF
       throw new IllegalArgumentException("value cannot be null");
     }
     if (!type.stored() && !type.indexed()) {
+      // nocommit... but it could be DocValue'd (only) tand
+      // that's ok...?
       throw new IllegalArgumentException("it doesn't make sense to have a field that "
         + "is neither indexed nor stored");
     }
-    if (!type.indexed() && !type.tokenized() && (type.storeTermVectors())) {
+    if (!type.indexed() && (type.storeTermVectors())) {
       throw new IllegalArgumentException("cannot store term vector information "
           + "for a field that is not indexed");
     }
@@ -132,6 +145,8 @@ public class Field implements IndexableF
     this.fieldsData = value;
   }
 
+  // nocommit Object getValue()?
+
   /**
    * The value of the field as a String, or null. If null, the Reader value or
    * binary value is used. Exactly one of stringValue(), readerValue(), and
@@ -202,6 +217,7 @@ public class Field implements IndexableF
    * Expert: change the value of this field. See <a
    * href="#setValue(java.lang.String)">setValue(String)</a>.
    */
+  // nocommit why not setValue(bytesref)...?
   public void setValue(byte[] value) {
     if (!isBinary()) {
       throw new IllegalArgumentException(
@@ -231,6 +247,8 @@ public class Field implements IndexableF
     return boost;
   }
 
+  // nocommit move set/getInt/Float/Long up here?  or just get/setNumber
+
   /** Sets the boost factor hits on this field.  This value will be
    * multiplied into the score of all hits on this this field of this
    * document.
@@ -292,20 +310,6 @@ public class Field implements IndexableF
     return result.toString();
   }
   
-  public void setDocValue(DocValue docValue) {
-    this.docValue = docValue;
-  }
-
-  @Override
-  public DocValue docValue() {
-    return null;
-  }
-  
-  @Override
-  public DocValues.Type docValueType() {
-    return null;
-  }
-
   /** Returns FieldType for this field. */
   public IndexableFieldType fieldType() {
     return type;

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/FieldType.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/FieldType.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/FieldType.java Wed Jan 11 16:51:55 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexableFieldType;
 
@@ -30,6 +31,7 @@ public class FieldType implements Indexa
   private boolean storeTermVectorPositions;
   private boolean omitNorms;
   private IndexOptions indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
+  private DocValues.Type docValueType;
   private boolean frozen;
 
   public FieldType(IndexableFieldType ref) {
@@ -134,6 +136,16 @@ public class FieldType implements Indexa
     this.indexOptions = value;
   }
 
+  public void setDocValueType(DocValues.Type type) {
+    checkIfFrozen();
+    docValueType = type;
+  }
+  
+  @Override
+  public DocValues.Type docValueType() {
+    return docValueType;
+  }
+
   /** Prints a Field for human consumption. */
   @Override
   public final String toString() {
@@ -173,6 +185,9 @@ public class FieldType implements Indexa
         result.append(indexOptions);
       }
     }
+    if (docValueType != null) {
+      result.append(",docValueType=" + docValueType);
+    }
     
     return result.toString();
   }

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/StringField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/StringField.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/StringField.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/document/StringField.java Wed Jan 11 16:51:55 2012
@@ -29,6 +29,9 @@ import org.apache.lucene.index.FieldInfo
  *  {@link StringField#TYPE_STORED} type (pass it to <code>new
  *  Field</code>) to store the value. */
 
+// nocommit maybe have a separate StoredField that apps add
+// to store a value; then String/TextField etc. wouldn't store...
+
 public final class StringField extends Field {
 
   /** Indexed, not tokenized, omits norms, indexes

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Wed Jan 11 16:51:55 2012
@@ -26,11 +26,9 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.codecs.PerDocConsumer;
 import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IOUtils;
@@ -82,17 +80,19 @@ final class DocFieldProcessor extends Do
     fieldsWriter.flush(state);
     consumer.flush(childFields, state);
 
+    for (DocValuesConsumerAndDocID consumer : docValues.values()) {
+      consumer.docValuesConsumer.finish(state.numDocs);
+    }
+
     // Important to save after asking consumer to flush so
     // consumer can alter the FieldInfo* if necessary.  EG,
     // FreqProxTermsWriter does this with
     // FieldInfo.storePayload.
     FieldInfosWriter infosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
     infosWriter.write(state.directory, state.segmentName, state.fieldInfos, IOContext.DEFAULT);
-    for (DocValuesConsumerAndDocID consumers : docValues.values()) {
-      consumers.docValuesConsumer.finish(state.numDocs);
-    }
+
     // close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
-    IOUtils.close(perDocConsumers.values());
+    IOUtils.close(perDocConsumer);
   }
 
   @Override
@@ -112,7 +112,7 @@ final class DocFieldProcessor extends Do
         field = next;
       }
     }
-    IOUtils.closeWhileHandlingException(perDocConsumers.values());
+    IOUtils.closeWhileHandlingException(perDocConsumer);
     // TODO add abort to PerDocConsumer!
     
     try {
@@ -132,7 +132,6 @@ final class DocFieldProcessor extends Do
     }
     
     try {
-      PerDocConsumer perDocConsumer = perDocConsumers.get(0);
       if (perDocConsumer != null) {
         perDocConsumer.abort();  
       }
@@ -176,7 +175,7 @@ final class DocFieldProcessor extends Do
     fieldHash = new DocFieldProcessorPerField[2];
     hashMask = 1;
     totalFieldCount = 0;
-    perDocConsumers.clear();
+    perDocConsumer = null;
     docValues.clear();
   }
 
@@ -270,9 +269,9 @@ final class DocFieldProcessor extends Do
       if (field.fieldType().stored()) {
         fieldsWriter.addField(field, fp.fieldInfo);
       }
-      final DocValue docValue = field.docValue();
-      if (docValue != null) {
-        docValuesConsumer(field.docValueType(), docState, fp.fieldInfo).add(docState.docID, docValue);
+      final DocValues.Type dvType = field.fieldType().docValueType();
+      if (dvType != null) {
+        docValuesConsumer(dvType, docState, fp.fieldInfo).add(docState.docID, field);
       }
     }
 
@@ -310,6 +309,8 @@ final class DocFieldProcessor extends Do
   }
 
   private static class DocValuesConsumerAndDocID {
+    // Only used to enforce that same DV field name is never
+    // added more than once per doc:
     public int docID;
     final DocValuesConsumer docValuesConsumer;
 
@@ -319,7 +320,7 @@ final class DocFieldProcessor extends Do
   }
 
   final private Map<String, DocValuesConsumerAndDocID> docValues = new HashMap<String, DocValuesConsumerAndDocID>();
-  final private Map<Integer, PerDocConsumer> perDocConsumers = new HashMap<Integer, PerDocConsumer>();
+  private PerDocConsumer perDocConsumer;
 
   DocValuesConsumer docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo) 
       throws IOException {
@@ -333,12 +334,9 @@ final class DocFieldProcessor extends Do
       return docValuesConsumerAndDocID.docValuesConsumer;
     }
 
-    PerDocConsumer perDocConsumer = perDocConsumers.get(0);
     if (perDocConsumer == null) {
       PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState("");
-      DocValuesFormat dvFormat = docState.docWriter.codec.docValuesFormat();
-      perDocConsumer = dvFormat.docsConsumer(perDocWriteState);
-      perDocConsumers.put(0, perDocConsumer);
+      perDocConsumer = docState.docWriter.codec.docValuesFormat().docsConsumer(perDocWriteState);
     }
     DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
     fieldInfo.setDocValuesType(valueType);

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocValue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocValue.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocValue.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/DocValue.java Wed Jan 11 16:51:55 2012
@@ -1,53 +0,0 @@
-package org.apache.lucene.index;
-
-/**
- * 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.util.Comparator;
-
-import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.document.DocValuesField;
-import org.apache.lucene.util.BytesRef;
-
-/**
- * Per document and field values consumed by {@link DocValuesConsumer}. 
- * @see DocValuesField
- * 
- * @lucene.experimental
- */
-public interface DocValue {
-
-  /**
-   * Returns the set {@link BytesRef} or <code>null</code> if not set.
-   */
-  public BytesRef getBytes();
-
-  /**
-   * Returns the set {@link BytesRef} comparator or <code>null</code> if not set
-   */
-  public Comparator<BytesRef> bytesComparator();
-
-  /**
-   * Returns the set floating point value or <code>0.0d</code> if not set.
-   */
-  public double getFloat();
-
-  /**
-   * Returns the set <code>long</code> value of <code>0</code> if not set.
-   */
-  public long getInt();
-
-}

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableField.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableField.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableField.java Wed Jan 11 16:51:55 2012
@@ -23,7 +23,6 @@ import java.io.Reader;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.NumericField;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.util.BytesRef;
 
 // TODO: how to handle versioning here...?
@@ -47,7 +46,7 @@ public interface IndexableField {
 
   /** Field boost (you must pre-multiply in any doc boost). */
   public float boost();
-  
+
   /** Non-null if this field has a binary value */
   public BytesRef binaryValue();
 
@@ -57,17 +56,16 @@ public interface IndexableField {
   /** Non-null if this field has a Reader value */
   public Reader readerValue();
 
-  // Numeric field:
-  /** True if this field is numeric */
-  public boolean numeric();
+  /** Numeric value; only used if the field is numeric
+   * (returns non-null result from {@link
+   * #numericDataType}. */
+  public Number numericValue();
 
   /** Numeric {@link org.apache.lucene.document.NumericField.DataType}; only used if
    * the field is numeric */
+  // nocommit move to IFT...?
   public NumericField.DataType numericDataType();
 
-  /** Numeric value; only used if the field is numeric */
-  public Number numericValue();
-
   /**
    * Returns the IndexableFieldType describing the properties of this field
    *
@@ -75,12 +73,6 @@ public interface IndexableField {
    */
   public IndexableFieldType fieldType();
   
-  /** Non-null if doc values should be indexed */
-  public DocValue docValue();
-
-  /** DocValues type; only used if docValue is non-null */
-  public DocValues.Type docValueType();
-
   /**
    * Creates the TokenStream used for indexing this field.  If appropriate,
    * implementations should use the given Analyzer to create the TokenStreams.

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableFieldType.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableFieldType.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/IndexableFieldType.java Wed Jan 11 16:51:55 2012
@@ -46,4 +46,8 @@ public interface IndexableFieldType {
   /** {@link IndexOptions}, describing what should be
    * recorded into the inverted index */
   public IndexOptions indexOptions();
+
+  /** DocValues type; if non-null then the field's value
+   *  will be indexed into docValues */
+  public DocValues.Type docValueType();
 }

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumer.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumer.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumer.java Wed Jan 11 16:51:55 2012
@@ -39,11 +39,9 @@ import org.apache.lucene.util.IOUtils;
 final class NormsConsumer extends InvertedDocEndConsumer {
   private final NormsFormat normsFormat;
   private PerDocConsumer consumer;
-  private final DocumentsWriterPerThread dwpt;
   
   public NormsConsumer(DocumentsWriterPerThread dwpt) {
     normsFormat = dwpt.codec.normsFormat();
-    this.dwpt = dwpt;
   }
 
   @Override
@@ -75,8 +73,8 @@ final class NormsConsumer extends Invert
             } else if (fi.isIndexed) {
               anythingFlushed = true;
               final DocValuesConsumer valuesConsumer = newConsumer(new PerDocWriteState(state), fi);
-              final DocValuesField value = new DocValuesField("");
-              value.setBytes(new BytesRef(new byte[] {0x00}), Type.BYTES_FIXED_STRAIGHT);
+              final DocValuesField value = new DocValuesField("", Type.BYTES_FIXED_STRAIGHT);
+              value.setBytes(new BytesRef(new byte[] {0x00}));
               valuesConsumer.add(state.numDocs-1, value);
               valuesConsumer.finish(state.numDocs);
             }

Modified: lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/java/org/apache/lucene/index/NormsConsumerPerField.java Wed Jan 11 16:51:55 2012
@@ -29,7 +29,7 @@ public class NormsConsumerPerField exten
   private final Similarity similarity;
   private final FieldInvertState fieldState;
   private DocValuesConsumer consumer;
-  private final DocValuesField value = new DocValuesField("");
+  private final DocValuesField value = new DocValuesField("", Type.BYTES_FIXED_STRAIGHT);
   private final BytesRef spare = new BytesRef(1);
   private final NormsConsumer parent;
   
@@ -53,9 +53,8 @@ public class NormsConsumerPerField exten
     if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
       DocValuesConsumer consumer = getConsumer();
       spare.bytes[0] = similarity.computeNorm(fieldState);
-      value.setBytes(spare, Type.BYTES_FIXED_STRAIGHT);
+      value.setBytes(spare);
       consumer.add(docState.docID, value);
-      
     }    
   }
   

Modified: lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexNormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexNormsConsumer.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexNormsConsumer.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexNormsConsumer.java Wed Jan 11 16:51:55 2012
@@ -22,12 +22,12 @@ import java.util.Arrays;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.PerDocConsumer;
-import org.apache.lucene.index.DocValue;
-import org.apache.lucene.index.DocValues;
 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.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -133,8 +133,8 @@ class PreFlexNormsConsumer extends PerDo
     }
     
     @Override
-    public void add(int docID, DocValue docValue) throws IOException {
-      add(docID, docValue.getBytes());
+    public void add(int docID, IndexableField docValue) throws IOException {
+      add(docID, docValue.binaryValue());
     }
     
     protected void add(int docID, BytesRef value) throws IOException {

Modified: lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/index/RandomIndexWriter.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/test-framework/java/org/apache/lucene/index/RandomIndexWriter.java Wed Jan 11 16:51:55 2012
@@ -28,7 +28,6 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.document.DocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexWriter; // javadoc
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
@@ -172,7 +171,7 @@ public class RandomIndexWriter implement
     String name = "random_" + type.name() + "" + docValuesFieldPrefix;
     if ("Lucene3x".equals(codec.getName()) || doc.getField(name) != null)
         return;
-    DocValuesField docValuesField = new DocValuesField(name);
+    DocValuesField docValuesField = new DocValuesField(name, type);
     switch (type) {
     case BYTES_FIXED_DEREF:
     case BYTES_FIXED_STRAIGHT:
@@ -186,13 +185,13 @@ public class RandomIndexWriter implement
         fixedRef.grow(fixedBytesLength);
         fixedRef.length = fixedBytesLength;
       }
-      docValuesField.setBytes(fixedRef, type);
+      docValuesField.setBytes(fixedRef);
       break;
     case BYTES_VAR_DEREF:
     case BYTES_VAR_STRAIGHT:
     case BYTES_VAR_SORTED:
       BytesRef ref = new BytesRef(_TestUtil.randomUnicodeString(random, 200));
-      docValuesField.setBytes(ref, type);
+      docValuesField.setBytes(ref);
       break;
     case FLOAT_32:
       docValuesField.setFloat(random.nextFloat());

Modified: lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/codecs/lucene40/TestDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/codecs/lucene40/TestDocValues.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/codecs/lucene40/TestDocValues.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/codecs/lucene40/TestDocValues.java Wed Jan 11 16:51:55 2012
@@ -18,17 +18,22 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import java.io.IOException;
+import java.io.Reader;
 import java.util.Comparator;
 
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.codecs.DocValuesConsumer;
 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.DocValue;
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.document.NumericField;
 import org.apache.lucene.index.DocValues.SortedSource;
 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.index.IndexableFieldType;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Counter;
@@ -175,9 +180,9 @@ public class TestDocValues extends Lucen
       Directory dir = newDirectory();
       final Counter trackBytes = Counter.newCounter();
       DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.VAR_INTS, newIOContext(random));
-      valueHolder.intValue = minMax[i][0];
+      valueHolder.numberValue = minMax[i][0];
       w.add(0, valueHolder);
-      valueHolder.intValue = minMax[i][1];
+      valueHolder.numberValue = minMax[i][1];
       w.add(1, valueHolder);
       w.finish(2);
       assertEquals(0, trackBytes.get());
@@ -212,7 +217,7 @@ public class TestDocValues extends Lucen
     final Counter trackBytes = Counter.newCounter();
     DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_8, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
-      valueHolder.intValue = (long) sourceArray[i];
+      valueHolder.numberValue = (long) sourceArray[i];
       w.add(i, valueHolder);
     }
     w.finish(sourceArray.length);
@@ -235,7 +240,7 @@ public class TestDocValues extends Lucen
     final Counter trackBytes = Counter.newCounter();
     DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_16, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
-      valueHolder.intValue = (long) sourceArray[i];
+      valueHolder.numberValue = (long) sourceArray[i];
       w.add(i, valueHolder);
     }
     w.finish(sourceArray.length);
@@ -258,7 +263,7 @@ public class TestDocValues extends Lucen
     final Counter trackBytes = Counter.newCounter();
     DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_64, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
-      valueHolder.intValue = sourceArray[i];
+      valueHolder.numberValue = sourceArray[i];
       w.add(i, valueHolder);
     }
     w.finish(sourceArray.length);
@@ -281,7 +286,7 @@ public class TestDocValues extends Lucen
     final Counter trackBytes = Counter.newCounter();
     DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_32, newIOContext(random));
     for (int i = 0; i < sourceArray.length; i++) {
-      valueHolder.intValue = (long) sourceArray[i];
+      valueHolder.numberValue = (long) sourceArray[i];
       w.add(i, valueHolder);
     }
     w.finish(sourceArray.length);
@@ -304,7 +309,7 @@ public class TestDocValues extends Lucen
     final Counter trackBytes = Counter.newCounter();
     DocValuesConsumer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), Type.FLOAT_32);
     for (int i = 0; i < sourceArray.length; i++) {
-      valueHolder.floatValue = sourceArray[i];
+      valueHolder.numberValue = sourceArray[i];
       w.add(i, valueHolder);
     }
     w.finish(sourceArray.length);
@@ -327,7 +332,7 @@ public class TestDocValues extends Lucen
     final Counter trackBytes = Counter.newCounter();
     DocValuesConsumer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), Type.FLOAT_64);
     for (int i = 0; i < sourceArray.length; i++) {
-      valueHolder.floatValue = sourceArray[i];
+      valueHolder.numberValue = sourceArray[i];
       w.add(i, valueHolder);
     }
     w.finish(sourceArray.length);
@@ -354,7 +359,7 @@ public class TestDocValues extends Lucen
       DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, type, newIOContext(random));
       for (int i = 0; i < NUM_VALUES; i++) {
         final long v = random.nextLong() % (1 + maxV);
-        valueHolder.intValue = values[i] = v;
+        valueHolder.numberValue = values[i] = v;
         w.add(i, valueHolder);
       }
       final int additionalDocs = 1 + random.nextInt(9);
@@ -390,7 +395,7 @@ public class TestDocValues extends Lucen
     for (int i = 0; i < NUM_VALUES; i++) {
       final double v = type == Type.FLOAT_32 ? random.nextFloat() : random
           .nextDouble();
-      valueHolder.floatValue = values[i] = v;
+      valueHolder.numberValue = values[i] = v;
       w.add(i, valueHolder);
     }
     final int additionalValues = 1 + random.nextInt(10);
@@ -431,31 +436,61 @@ public class TestDocValues extends Lucen
     return getSource(values).asSortedSource();
   }
   
-  public static class DocValueHolder implements DocValue {
+  public static class DocValueHolder implements IndexableField {
     BytesRef bytes;
-    long intValue;
-    double floatValue;
+    Number numberValue;
     Comparator<BytesRef> comp;
+
+    @Override
+    public TokenStream tokenStream(Analyzer a) {
+      return null;
+    }
+
+    @Override
+    public float boost() {
+      return 0.0f;
+    }
+
+    @Override
+    public String name() {
+      return "test";
+    }
+
     @Override
-    public BytesRef getBytes() {
+    public BytesRef binaryValue() {
       return bytes;
     }
 
+    /*
     @Override
     public Comparator<BytesRef> bytesComparator() {
       return comp;
     }
+    */
+
+    @Override
+    public Number numericValue() {
+      return numberValue;
+    }
 
     @Override
-    public double getFloat() {
-      return floatValue;
+    public String stringValue() {
+      return null;
     }
 
     @Override
-    public long getInt() {
-      return intValue;
+    public Reader readerValue() {
+      return null;
+    }
+
+    @Override
+    public IndexableFieldType fieldType() {
+      return null;
+    }
+
+    @Override
+    public NumericField.DataType numericDataType() {
+      return null;
     }
-    
   }
-  
 }

Modified: lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Wed Jan 11 16:51:55 2012
@@ -1256,7 +1256,7 @@ public class TestAddIndexes extends Luce
     RandomIndexWriter w = new RandomIndexWriter(random, d1);
     Document doc = new Document();
     doc.add(newField("id", "1", StringField.TYPE_STORED));
-    DocValuesField dv = new DocValuesField("dv");
+    DocValuesField dv = new DocValuesField("dv", DocValues.Type.VAR_INTS);
     dv.setInt(1);
     doc.add(dv);
     w.addDocument(doc);
@@ -1267,7 +1267,7 @@ public class TestAddIndexes extends Luce
     w = new RandomIndexWriter(random, d2);
     doc = new Document();
     doc.add(newField("id", "2", StringField.TYPE_STORED));
-    dv = new DocValuesField("dv");
+    dv = new DocValuesField("dv", DocValues.Type.VAR_INTS);
     dv.setInt(2);
     doc.add(dv);
     w.addDocument(doc);

Modified: lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDocValuesIndexing.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDocValuesIndexing.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDocValuesIndexing.java Wed Jan 11 16:51:55 2012
@@ -24,7 +24,6 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -37,17 +36,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.DocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValues.SortedSource;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LogDocMergePolicy;
-import org.apache.lucene.index.LogMergePolicy;
-import org.apache.lucene.index.MultiDocValues;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.search.*;
@@ -85,7 +74,7 @@ public class TestDocValuesIndexing exten
     IndexWriter writer = new IndexWriter(dir, writerConfig(false));
     for (int i = 0; i < 5; i++) {
       Document doc = new Document();
-      DocValuesField valuesField = new DocValuesField("docId");
+      DocValuesField valuesField = new DocValuesField("docId", DocValues.Type.VAR_INTS);
       valuesField.setInt(i);
       doc.add(valuesField);
       doc.add(new TextField("docId", "" + i));
@@ -582,7 +571,7 @@ public class TestDocValuesIndexing exten
     final boolean isNumeric = NUMERICS.contains(value);
     FixedBitSet deleted = new FixedBitSet(numValues);
     Document doc = new Document();
-    DocValuesField valField = new DocValuesField(value.name());
+    DocValuesField valField = new DocValuesField(value.name(), value);
     doc.add(valField);
     final BytesRef bytesRef = new BytesRef();
 
@@ -601,16 +590,16 @@ public class TestDocValuesIndexing exten
           valField.setInt((long)i);
           break;
         case FIXED_INTS_16:
-          valField.setInt((short)i, random.nextInt(10) != 0);
+          valField.setInt((short)i);
           break;
         case FIXED_INTS_32:
-          valField.setInt(i, random.nextInt(10) != 0);
+          valField.setInt(i);
           break;
         case FIXED_INTS_64:
-          valField.setInt((long)i, random.nextInt(10) != 0);
+          valField.setInt((long)i);
           break;
         case FIXED_INTS_8:
-          valField.setInt((byte)(0xFF & (i % 128)), random.nextInt(10) != 0);
+          valField.setInt((byte)(0xFF & (i % 128)));
           break;
         case FLOAT_32:
           valField.setFloat(2.0f * i);
@@ -627,7 +616,7 @@ public class TestDocValuesIndexing exten
           b[j] = upto++;
         }
         if (bytesRef != null) {
-          valField.setBytes(bytesRef, value);
+          valField.setBytes(bytesRef);
         }
       }
       doc.removeFields("id");
@@ -663,7 +652,7 @@ public class TestDocValuesIndexing exten
     Directory d = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random, d);
     Document doc = new Document();
-    DocValuesField f = new DocValuesField("field");
+    DocValuesField f = new DocValuesField("field", Type.VAR_INTS);
     f.setInt(17);
     // Index doc values are single-valued so we should not
     // be able to add same field more than once:
@@ -691,12 +680,12 @@ public class TestDocValuesIndexing exten
     Directory d = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random, d);
     Document doc = new Document();
-    DocValuesField f = new DocValuesField("field");
+    DocValuesField f = new DocValuesField("field", Type.VAR_INTS);
     f.setInt(17);
     // Index doc values are single-valued so we should not
     // be able to add same field more than once:
     doc.add(f);
-    DocValuesField f2 = new DocValuesField("field");
+    DocValuesField f2 = new DocValuesField("field", Type.FLOAT_32);
     f2.setFloat(22.0);
     doc.add(f2);
     try {
@@ -725,6 +714,7 @@ public class TestDocValuesIndexing exten
       IndexWriterConfig cfg = newIndexWriterConfig(TEST_VERSION_CURRENT,
           new MockAnalyzer(random));
       IndexWriter w = new IndexWriter(d, cfg);
+      // nocommit
       Comparator<BytesRef> comp = BytesRef.getUTF8SortedAsUnicodeComparator();
       int numDocs = atLeast(100);
       BytesRefHash hash = new BytesRefHash();
@@ -733,13 +723,13 @@ public class TestDocValuesIndexing exten
       for (int i = 0; i < numDocs; i++) {
         Document doc = new Document();
         doc.add(newField("id", "" + i, TextField.TYPE_STORED));
-        DocValuesField f = new DocValuesField("field");
+        DocValuesField f = new DocValuesField("field", type);
         String string =fixed ? _TestUtil.randomFixedByteLengthUnicodeString(random,
             len) : _TestUtil.randomRealisticUnicodeString(random, 1, len);
         hash.add(new BytesRef(string));
         docToString.put("" + i, string);
 
-        f.setBytes(new BytesRef(string), type, comp);
+        f.setBytes(new BytesRef(string));
         doc.add(f);
         w.addDocument(doc);
       }
@@ -763,12 +753,12 @@ public class TestDocValuesIndexing exten
         Document doc = new Document();
         String id = "" + i + numDocs;
         doc.add(newField("id", id, TextField.TYPE_STORED));
-        DocValuesField f = new DocValuesField("field");
+        DocValuesField f = new DocValuesField("field", type);
         String string = fixed ? _TestUtil.randomFixedByteLengthUnicodeString(random,
             len) : _TestUtil.randomRealisticUnicodeString(random, 1, len);
         hash.add(new BytesRef(string));
         docToString.put(id, string);
-        f.setBytes(new BytesRef(string), type, comp);
+        f.setBytes(new BytesRef(string));
         doc.add(f);
         w.addDocument(doc);
       }

Modified: lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDuelingCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDuelingCodecs.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDuelingCodecs.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestDuelingCodecs.java Wed Jan 11 16:51:55 2012
@@ -500,7 +500,6 @@ public class TestDuelingCodecs extends L
     assertEquals(info, leftField.binaryValue(), rightField.binaryValue());
     assertEquals(info, leftField.stringValue(), rightField.stringValue());
     assertEquals(info, leftField.numericValue(), rightField.numericValue());
-    assertEquals(info, leftField.numeric(), rightField.numeric());
     assertEquals(info, leftField.numericDataType(), rightField.numericDataType());
     // TODO: should we check the FT at all?
   }

Modified: lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestIndexableField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestIndexableField.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestIndexableField.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestIndexableField.java Wed Jan 11 16:51:55 2012
@@ -28,7 +28,6 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.NumericField.DataType;
 import org.apache.lucene.document.NumericField;
 import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
@@ -85,6 +84,11 @@ public class TestIndexableField extends 
       public FieldInfo.IndexOptions indexOptions() {
         return FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
       }
+
+      @Override
+      public DocValues.Type docValueType() {
+        return null;
+      }
     };
 
     public MyField(int counter) {
@@ -135,13 +139,8 @@ public class TestIndexableField extends 
 
     // Numeric field:
     @Override
-    public boolean numeric() {
-      return counter%10 == 9;
-    }
-
-    @Override
     public DataType numericDataType() {
-      return DataType.INT;
+      return counter%10 == 9 ? DataType.INT : null;
     }
 
     @Override
@@ -154,24 +153,14 @@ public class TestIndexableField extends 
       return fieldType;
     }
 
-    // TODO: randomly enable doc values
-    @Override
-    public DocValue docValue() {
-      return null;
-    }
-
-    @Override
-    public DocValues.Type docValueType() {
-      return null;
-    }
-
     @Override
     public TokenStream tokenStream(Analyzer analyzer) throws IOException {
-      if (numeric()) {
+      if (numericDataType() != null) {
         return new NumericField(name()).setIntValue(counter).tokenStream(analyzer);
-      }
-      return readerValue() != null ? analyzer.tokenStream(name(), readerValue()) :
+      } else {
+        return readerValue() != null ? analyzer.tokenStream(name(), readerValue()) :
           analyzer.tokenStream(name(), new StringReader(stringValue()));
+      }
     }
   }
 

Modified: lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestTypePromotion.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestTypePromotion.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestTypePromotion.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/index/TestTypePromotion.java Wed Jan 11 16:51:55 2012
@@ -11,16 +11,9 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.DocValuesField;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.ReaderContext;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.NoMergePolicy;
-import org.apache.lucene.index.SlowMultiReaderWrapper;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
@@ -81,11 +74,11 @@ public class TestTypePromotion extends L
     int num_2 = atLeast(200);
     int num_3 = atLeast(200);
     long[] values = new long[num_1 + num_2 + num_3];
-    index(writer, new DocValuesField("promote"),
+    index(writer,
         randomValueType(types, random), values, 0, num_1);
     writer.commit();
     
-    index(writer, new DocValuesField("promote"),
+    index(writer,
         randomValueType(types, random), values, num_1, num_2);
     writer.commit();
     
@@ -96,7 +89,7 @@ public class TestTypePromotion extends L
       Directory dir_2 = newDirectory() ;
       IndexWriter writer_2 = new IndexWriter(dir_2,
           newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
-      index(writer_2, new DocValuesField("promote"),
+      index(writer_2,
           randomValueType(types, random), values, num_1 + num_2, num_3);
       writer_2.commit();
       writer_2.close();
@@ -110,7 +103,7 @@ public class TestTypePromotion extends L
       }
       dir_2.close();
     } else {
-      index(writer, new DocValuesField("promote"),
+      index(writer,
           randomValueType(types, random), values, num_1 + num_2, num_3);
     }
 
@@ -172,9 +165,10 @@ public class TestTypePromotion extends L
     reader.close();
   }
 
-  public void index(IndexWriter writer, DocValuesField valField,
+  public void index(IndexWriter writer,
       Type valueType, long[] values, int offset, int num)
       throws CorruptIndexException, IOException {
+    DocValuesField valField =  new DocValuesField("promote", valueType);
     BytesRef ref = new BytesRef(new byte[] { 1, 2, 3, 4 });
     for (int i = offset; i < offset + num; i++) {
       Document doc = new Document();
@@ -186,15 +180,15 @@ public class TestTypePromotion extends L
         break;
       case FIXED_INTS_16:
         values[i] = random.nextInt(Short.MAX_VALUE);
-        valField.setInt((short) values[i], true);
+        valField.setInt((short) values[i]);
         break;
       case FIXED_INTS_32:
         values[i] = random.nextInt();
-        valField.setInt((int) values[i], true);
+        valField.setInt((int) values[i]);
         break;
       case FIXED_INTS_64:
         values[i] = random.nextLong();
-        valField.setInt(values[i], true);
+        valField.setInt(values[i]);
         break;
       case FLOAT_64:
         double nextDouble = random.nextDouble();
@@ -208,14 +202,14 @@ public class TestTypePromotion extends L
         break;
       case FIXED_INTS_8:
          values[i] = (byte) i;
-        valField.setInt((byte)values[i], true);
+        valField.setInt((byte)values[i]);
         break;
       case BYTES_FIXED_DEREF:
       case BYTES_FIXED_SORTED:
       case BYTES_FIXED_STRAIGHT:
         values[i] = random.nextLong();
         BytesRefUtils.copyLong(ref, values[i]);
-        valField.setBytes(ref, valueType);
+        valField.setBytes(ref);
         break;
       case BYTES_VAR_DEREF:
       case BYTES_VAR_SORTED:
@@ -227,12 +221,11 @@ public class TestTypePromotion extends L
           BytesRefUtils.copyLong(ref, random.nextLong());
           values[i] = BytesRefUtils.asLong(ref);
         }
-        valField.setBytes(ref, valueType);
+        valField.setBytes(ref);
         break;
 
       default:
         fail("unexpected value " + valueType);
-
       }
       doc.add(valField);
       writer.addDocument(doc);
@@ -267,7 +260,7 @@ public class TestTypePromotion extends L
     int num_1 = atLeast(200);
     int num_2 = atLeast(200);
     long[] values = new long[num_1 + num_2];
-    index(writer, new DocValuesField("promote"),
+    index(writer,
         randomValueType(INTEGERS, random), values, 0, num_1);
     writer.commit();
     
@@ -276,7 +269,7 @@ public class TestTypePromotion extends L
       Directory dir_2 = newDirectory() ;
       IndexWriter writer_2 = new IndexWriter(dir_2,
           newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
-      index(writer_2, new DocValuesField("promote"),
+      index(writer_2,
           randomValueType(random.nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random), values, num_1, num_2);
       writer_2.commit();
       writer_2.close();
@@ -290,7 +283,7 @@ public class TestTypePromotion extends L
       }
       dir_2.close();
     } else {
-      index(writer, new DocValuesField("promote"),
+      index(writer,
           randomValueType(random.nextBoolean() ? UNSORTED_BYTES : SORTED_BYTES, random), values, num_1, num_2);
       writer.commit();
     }

Modified: lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java?rev=1230141&r1=1230140&r2=1230141&view=diff
==============================================================================
--- lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java (original)
+++ lucene/dev/branches/lucene3453/lucene/src/test/org/apache/lucene/search/TestDocValuesScoring.java Wed Jan 11 16:51:55 2012
@@ -20,16 +20,17 @@ package org.apache.lucene.search;
 import java.io.IOException;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.DocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.DocValuesField;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DocValues.Source;
+import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.DocValues.Source;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
@@ -54,7 +55,7 @@ public class TestDocValuesScoring extend
     Document doc = new Document();
     Field field = newField("foo", "", TextField.TYPE_UNSTORED);
     doc.add(field);
-    DocValuesField dvField = new DocValuesField("foo_boost");
+    DocValuesField dvField = new DocValuesField("foo_boost", DocValues.Type.FLOAT_32);
     doc.add(dvField);
     Field field2 = newField("bar", "", TextField.TYPE_UNSTORED);
     doc.add(field2);