You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/01/18 23:28:20 UTC

svn commit: r1233096 [5/13] - in /lucene/dev/branches/solrcloud: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ dev-tools/idea/modules/analysis/kuromoji/ dev-tools/idea/solr/contrib/analysis-extras/ dev-tools/maven/modules/a...

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java Wed Jan 18 22:28:07 2012
@@ -35,13 +35,15 @@ class SimpleTextFieldsWriter extends Fie
   private final IndexOutput out;
   private final BytesRef scratch = new BytesRef(10);
 
-  final static BytesRef END     = new BytesRef("END");
-  final static BytesRef FIELD   = new BytesRef("field ");
-  final static BytesRef TERM    = new BytesRef("  term ");
-  final static BytesRef DOC     = new BytesRef("    doc ");
-  final static BytesRef FREQ    = new BytesRef("      freq ");
-  final static BytesRef POS     = new BytesRef("      pos ");
-  final static BytesRef PAYLOAD = new BytesRef("        payload ");
+  final static BytesRef END          = new BytesRef("END");
+  final static BytesRef FIELD        = new BytesRef("field ");
+  final static BytesRef TERM         = new BytesRef("  term ");
+  final static BytesRef DOC          = new BytesRef("    doc ");
+  final static BytesRef FREQ         = new BytesRef("      freq ");
+  final static BytesRef POS          = new BytesRef("      pos ");
+  final static BytesRef START_OFFSET = new BytesRef("      startOffset ");
+  final static BytesRef END_OFFSET   = new BytesRef("      endOffset ");
+  final static BytesRef PAYLOAD      = new BytesRef("        payload ");
 
   public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
     final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentName, state.segmentSuffix);
@@ -97,10 +99,19 @@ class SimpleTextFieldsWriter extends Fie
   private class SimpleTextPostingsWriter extends PostingsConsumer {
     private BytesRef term;
     private boolean wroteTerm;
-    private IndexOptions indexOptions;
+    private final IndexOptions indexOptions;
+    private final boolean writePositions;
+    private final boolean writeOffsets;
+
+    // for assert:
+    private int lastEndOffset = -1;
 
     public SimpleTextPostingsWriter(FieldInfo field) {
       this.indexOptions = field.indexOptions;
+      writePositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+      writeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+      //System.out.println("writeOffsets=" + writeOffsets);
+      //System.out.println("writePos=" + writePositions);
     }
 
     @Override
@@ -121,10 +132,10 @@ class SimpleTextFieldsWriter extends Fie
         write(Integer.toString(termDocFreq));
         newline();
       }
+
+      lastEndOffset = -1;
     }
     
-    
-
     public PostingsConsumer reset(BytesRef term) {
       this.term = term;
       wroteTerm = false;
@@ -132,10 +143,25 @@ class SimpleTextFieldsWriter extends Fie
     }
 
     @Override
-    public void addPosition(int position, BytesRef payload) throws IOException {
-      write(POS);
-      write(Integer.toString(position));
-      newline();
+    public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
+      if (writePositions) {
+        write(POS);
+        write(Integer.toString(position));
+        newline();
+      }
+
+      if (writeOffsets) {
+        assert endOffset >= startOffset;
+        assert startOffset >= lastEndOffset: "startOffset=" + startOffset + " lastEndOffset=" + lastEndOffset;
+        lastEndOffset = endOffset;
+        write(START_OFFSET);
+        write(Integer.toString(startOffset));
+        newline();
+        write(END_OFFSET);
+        write(Integer.toString(endOffset));
+        newline();
+      }
+
       if (payload != null && payload.length > 0) {
         assert payload.length != 0;
         write(PAYLOAD);

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsConsumer.java Wed Jan 18 22:28:07 2012
@@ -17,18 +17,19 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
+import java.io.Closeable;
 import java.io.IOException;
 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;
@@ -71,7 +72,17 @@ public class SimpleTextNormsConsumer ext
   @Override
   public void close() throws IOException {
     if (writer != null) {
-      writer.finish();
+      boolean success = false;
+      try {
+        writer.finish();
+        success = true;
+      } finally {
+        if (success) {
+          IOUtils.close(writer);
+        } else {
+          IOUtils.closeWhileHandlingException(writer);
+        }
+      }
     }
   }
   
@@ -83,17 +94,20 @@ public class SimpleTextNormsConsumer ext
 
   @Override
   protected boolean canMerge(FieldInfo info) {
-    return !info.omitNorms && info.isIndexed;
+    return info.normsPresent();
   }
 
   @Override
   protected Type getDocValuesType(FieldInfo info) {
-    return Type.BYTES_FIXED_STRAIGHT;
+    return info.getNormType();
   }
 
   @Override
   public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)
       throws IOException {
+    if (type != Type.FIXED_INTS_8) {
+      throw new UnsupportedOperationException("Codec only supports single byte norm values. Type give: " + type);
+    }
     return new SimpleTextNormsDocValuesConsumer(fieldInfo);
   }
 
@@ -119,11 +133,11 @@ 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.numericValue().longValue());
     }
     
-    protected void add(int docID, BytesRef value) throws IOException {
+    public void add(int docID, long value) {
       if (docIDs.length <= upto) {
         assert docIDs.length == upto;
         docIDs = ArrayUtil.grow(docIDs, 1 + upto);
@@ -132,8 +146,8 @@ public class SimpleTextNormsConsumer ext
         assert norms.length == upto;
         norms = ArrayUtil.grow(norms, 1 + upto);
       }
-      assert value.length == 1;
-      norms[upto] = value.bytes[value.offset];
+      norms[upto] = (byte) value;
+      
       docIDs[upto] = docID;
       upto++;
     }
@@ -181,7 +195,7 @@ public class SimpleTextNormsConsumer ext
     return writer;
   }
 
-  private static class NormsWriter {
+  private static class NormsWriter implements Closeable{
 
     private final IndexOutput output;
     private int numTotalDocs = 0;
@@ -253,12 +267,16 @@ public class SimpleTextNormsConsumer ext
     }
 
     public void abort() throws IOException {
-      IOUtils.close(output);
+      close();
     }
 
     public void finish() throws IOException {
-      finish(numTotalDocs);
-      IOUtils.close(output);
+        finish(numTotalDocs);
+    }
+
+    @Override
+    public void close() throws IOException {
+      output.close();
     }
   }
 
@@ -266,7 +284,7 @@ public class SimpleTextNormsConsumer ext
     FieldInfos fieldInfos = info.getFieldInfos();
     
     for (FieldInfo fieldInfo : fieldInfos) {
-      if (!fieldInfo.omitNorms && fieldInfo.isIndexed) {
+      if (fieldInfo.normsPresent()) {
         files.add(IndexFileNames.segmentFileName(info.name, "",
             NORMS_EXTENSION));  
         break;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsProducer.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsProducer.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsProducer.java Wed Jan 18 22:28:07 2012
@@ -32,6 +32,7 @@ import org.apache.lucene.codecs.PerDocPr
 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.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
@@ -95,11 +96,12 @@ public class SimpleTextNormsProducer ext
   }
   
   static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
-    // TODO: This is what SI always did... but we can do this cleaner?
-    // like first FI that has norms but doesn't have separate norms?
-    final String normsFileName = IndexFileNames.segmentFileName(info.name, "", SimpleTextNormsConsumer.NORMS_EXTENSION);
-    if (dir.fileExists(normsFileName)) {
-      files.add(normsFileName);
+    FieldInfos fieldInfos = info.getFieldInfos();
+    for (FieldInfo fieldInfo : fieldInfos) {
+      if (fieldInfo.normsPresent()) {
+        files.add(IndexFileNames.segmentFileName(info.name, "", SimpleTextNormsConsumer.NORMS_EXTENSION));
+        break;
+      }
     }
   }
   
@@ -130,7 +132,7 @@ public class SimpleTextNormsProducer ext
 
     @Override
     public Type type() {
-      return Type.BYTES_FIXED_STRAIGHT;
+      return Type.FIXED_INTS_8;
     }
 
     @Override
@@ -141,7 +143,7 @@ public class SimpleTextNormsProducer ext
   
   static final class Norm extends Source {
     protected Norm(byte[] bytes) {
-      super(Type.BYTES_FIXED_STRAIGHT);
+      super(Type.FIXED_INTS_8);
       this.bytes = bytes;
     }
     final byte bytes[];
@@ -153,6 +155,11 @@ public class SimpleTextNormsProducer ext
       ref.length = 1;
       return ref;
     }
+    
+    @Override
+    public long getInt(int docID) {
+      return bytes[docID];
+    }
 
     @Override
     public boolean hasArray() {

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java Wed Jan 18 22:28:07 2012
@@ -98,46 +98,39 @@ public class SimpleTextStoredFieldsWrite
     newLine();
     
     write(TYPE);
-    if (field.numeric()) {
-      switch (field.numericDataType()) {
-        case INT:
-          write(TYPE_INT);
-          newLine();
-          
-          write(VALUE);
-          write(Integer.toString(field.numericValue().intValue()));
-          newLine();
-          
-          break;
-        case LONG:
-          write(TYPE_LONG);
-          newLine();
-          
-          write(VALUE);
-          write(Long.toString(field.numericValue().longValue()));
-          newLine();
-          
-          break;
-        case FLOAT:
-          write(TYPE_FLOAT);
-          newLine();
-          
-          write(VALUE);
-          write(Float.toString(field.numericValue().floatValue()));
-          newLine();
+    final Number n = field.numericValue();
+
+    if (n != null) {
+      if (n instanceof Byte || n instanceof Short || n instanceof Integer) {
+        write(TYPE_INT);
+        newLine();
           
-          break;
-        case DOUBLE:
-          write(TYPE_DOUBLE);
-          newLine();
+        write(VALUE);
+        write(Integer.toString(n.intValue()));
+        newLine();
+      } else if (n instanceof Long) {
+        write(TYPE_LONG);
+        newLine();
+
+        write(VALUE);
+        write(Long.toString(n.longValue()));
+        newLine();
+      } else if (n instanceof Float) {
+        write(TYPE_FLOAT);
+        newLine();
           
-          write(VALUE);
-          write(Double.toString(field.numericValue().doubleValue()));
-          newLine();
+        write(VALUE);
+        write(Float.toString(n.floatValue()));
+        newLine();
+      } else if (n instanceof Double) {
+        write(TYPE_DOUBLE);
+        newLine();
           
-          break;
-        default:
-          assert false : "Should never get here";
+        write(VALUE);
+        write(Double.toString(n.doubleValue()));
+        newLine();
+      } else {
+        throw new IllegalArgumentException("cannot store numeric type " + n.getClass());
       }
     } else { 
       BytesRef bytes = field.binaryValue();

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsFormat.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsFormat.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsFormat.java Wed Jan 18 22:28:07 2012
@@ -38,7 +38,7 @@ public class SimpleTextTermVectorsFormat
 
   @Override
   public TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
-    return new SimpleTextTermVectorsReader(directory, segmentInfo, fieldInfos, context);
+    return new SimpleTextTermVectorsReader(directory, segmentInfo, context);
   }
 
   @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Wed Jan 18 22:28:07 2012
@@ -26,11 +26,9 @@ import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexFileNames;
@@ -63,7 +61,7 @@ public class SimpleTextTermVectorsReader
   private BytesRef scratch = new BytesRef();
   private CharsRef scratchUTF16 = new CharsRef();
   
-  public SimpleTextTermVectorsReader(Directory directory, SegmentInfo si, FieldInfos fieldInfos, IOContext context) throws IOException {
+  public SimpleTextTermVectorsReader(Directory directory, SegmentInfo si, IOContext context) throws IOException {
     boolean success = false;
     try {
       in = directory.openInput(IndexFileNames.segmentFileName(si.name, "", VECTORS_EXTENSION), context);
@@ -114,7 +112,8 @@ public class SimpleTextTermVectorsReader
     for (int i = 0; i < numFields; i++) {
       readLine();
       assert StringHelper.startsWith(scratch, FIELD);
-      int fieldNumber = parseIntAt(FIELD.length);
+      // skip fieldNumber:
+      parseIntAt(FIELD.length);
       
       readLine();
       assert StringHelper.startsWith(scratch, FIELDNAME);
@@ -373,13 +372,16 @@ public class SimpleTextTermVectorsReader
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
       SimpleTVPostings postings = current.getValue();
       if (postings.positions == null && postings.startOffsets == null) {
         return null;
       }
+      if (needsOffsets && (postings.startOffsets == null || postings.endOffsets == null)) {
+        return null;
+      }
       // TODO: reuse
-      SimpleTVDocsAndPositionsEnum e = new SimpleTVDocsAndPositionsEnum(postings.startOffsets != null);
+      SimpleTVDocsAndPositionsEnum e = new SimpleTVDocsAndPositionsEnum();
       e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets);
       return e;
     }
@@ -436,7 +438,6 @@ public class SimpleTextTermVectorsReader
   }
   
   private static class SimpleTVDocsAndPositionsEnum extends DocsAndPositionsEnum {
-    private final OffsetAttribute offsetAtt;
     private boolean didNext;
     private int doc = -1;
     private int nextPos;
@@ -445,18 +446,6 @@ public class SimpleTextTermVectorsReader
     private int[] startOffsets;
     private int[] endOffsets;
 
-    public SimpleTVDocsAndPositionsEnum(boolean storeOffsets) {
-      if (storeOffsets) {
-        offsetAtt = attributes().addAttribute(OffsetAttribute.class);
-      } else {
-        offsetAtt = null;
-      }
-    }
-
-    public boolean canReuse(boolean storeOffsets) {
-      return storeOffsets == (offsetAtt != null);
-    }
-
     @Override
     public int freq() {
       if (positions != null) {
@@ -495,7 +484,6 @@ public class SimpleTextTermVectorsReader
       this.liveDocs = liveDocs;
       this.positions = positions;
       this.startOffsets = startOffsets;
-      assert (offsetAtt != null) == (startOffsets != null);
       this.endOffsets = endOffsets;
       this.doc = -1;
       didNext = false;
@@ -516,11 +504,6 @@ public class SimpleTextTermVectorsReader
     public int nextPosition() {
       assert (positions != null && nextPos < positions.length) ||
         startOffsets != null && nextPos < startOffsets.length;
-
-      if (startOffsets != null) {
-        offsetAtt.setOffset(startOffsets[nextPos],
-                            endOffsets[nextPos]);
-      }
       if (positions != null) {
         return positions[nextPos++];
       } else {
@@ -528,5 +511,15 @@ public class SimpleTextTermVectorsReader
         return -1;
       }
     }
+
+    @Override
+    public int startOffset() {
+      return startOffsets[nextPos-1];
+    }
+
+    @Override
+    public int endOffset() {
+      return endOffsets[nextPos-1];
+    }
   }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java Wed Jan 18 22:28:07 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
+import java.util.Comparator;
 
 import org.apache.lucene.codecs.TermVectorsWriter;
 import org.apache.lucene.index.FieldInfo;
@@ -170,6 +171,11 @@ public class SimpleTextTermVectorsWriter
     }
   }
   
+  @Override
+  public Comparator<BytesRef> getComparator() throws IOException {
+    return BytesRef.getUTF8SortedAsUnicodeComparator();
+  }
+  
   private void write(String s) throws IOException {
     SimpleTextUtil.write(out, s, scratch);
   }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/DocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/DocValuesField.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/DocValuesField.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/DocValuesField.java Wed Jan 18 22:28:07 2012
@@ -16,13 +16,14 @@ package org.apache.lucene.document;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-import java.io.Reader;
+
 import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
 
-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.index.DocValues;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -32,14 +33,16 @@ import org.apache.lucene.util.BytesRef;
  * example usage, adding an int value:
  * 
  * <pre>
- * document.add(new DocValuesField(name).setInt(value));
+ * DocValuesField field = new DocValuesField(name, DocValues.Type.VAR_INTS);
+ * field.setInt(value);
+ * document.add(field);
  * </pre>
  * 
  * For optimal performance, re-use the <code>DocValuesField</code> and
  * {@link Document} instance for more than one document:
  * 
  * <pre>
- *  DocValuesField field = new DocValuesField(name);
+ *  DocValuesField field = new DocValuesField(name, DocValues.Type.VAR_INTS);
  *  Document document = new Document();
  *  document.add(field);
  * 
@@ -69,326 +72,79 @@ import org.apache.lucene.util.BytesRef;
  * </pre>
  * 
  * */
-public class DocValuesField extends Field implements DocValue {
-
-  protected BytesRef bytes;
-  protected double doubleValue;
-  protected long longValue;
-  protected DocValues.Type type;
-  protected Comparator<BytesRef> bytesComparator;
-
-  /**
-   * Creates a new {@link DocValuesField} with the given name.
-   */
-  public DocValuesField(String name) {
-    this(name, new FieldType());
-  }
-
-  public DocValuesField(String name, IndexableFieldType type) {
-    this(name, type, null);
-  }
 
-  public DocValuesField(String name, IndexableFieldType type, String value) {
-    super(name, type);
-    fieldsData = value;
-  }
+public class DocValuesField extends Field {
 
-  @Override
-  public DocValue docValue() {
-    return this;
-  }
+  protected Comparator<BytesRef> bytesComparator;
 
-  /**
-   * 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)}.
-   */
-  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;
+  private static final Map<DocValues.Type,FieldType> types = new HashMap<DocValues.Type,FieldType>();
+  static {
+    for(DocValues.Type type : DocValues.Type.values()) {
+      final FieldType ft = new FieldType();
+      ft.setDocValueType(type);
+      ft.freeze();
+      types.put(type, ft);
     }
-    longValue = 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)}.
-   */
-  public void setInt(int value) {
-    setInt(value, false);
-  }
+  private static EnumSet<Type> BYTES = EnumSet.of(
+                     Type.BYTES_FIXED_DEREF,
+                     Type.BYTES_FIXED_STRAIGHT,
+                     Type.BYTES_VAR_DEREF,
+                     Type.BYTES_VAR_STRAIGHT,
+                     Type.BYTES_FIXED_SORTED,
+                     Type.BYTES_VAR_SORTED);
 
-  /**
-   * 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;
-  }
+  private static EnumSet<Type> INTS = EnumSet.of(
+                     Type.VAR_INTS,
+                     Type.FIXED_INTS_8,
+                     Type.FIXED_INTS_16,
+                     Type.FIXED_INTS_32,
+                     Type.FIXED_INTS_64);
 
-  /**
-   * 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)}.
-   */
-  public void setInt(short value) {
-    setInt(value, false);
+  public static FieldType getFieldType(DocValues.Type type) {
+    return types.get(type);
   }
 
-  /**
-   * 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;
+  public DocValuesField(String name, BytesRef bytes, DocValues.Type docValueType) {
+    super(name, getFieldType(docValueType));
+    if (!BYTES.contains(docValueType)) {
+      throw new IllegalArgumentException("docValueType must be one of: " + BYTES + "; got " + docValueType);
     }
-    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)}.
-   */
-  public void setInt(byte value) {
-    setInt(value, false);
+    fieldsData = bytes;
   }
 
-  /**
-   * 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;
+  public DocValuesField(String name, int value, DocValues.Type docValueType) {
+    super(name, getFieldType(docValueType));
+    if (!INTS.contains(docValueType)) {
+      throw new IllegalArgumentException("docValueType must be one of: " + INTS +"; got " + docValueType);
     }
-    longValue = value;
+    fieldsData = Integer.valueOf(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)}.
-   */
-  public void setFloat(float value) {
-    if (type == null) {
-      type = DocValues.Type.FLOAT_32;
+  public DocValuesField(String name, long value, DocValues.Type docValueType) {
+    super(name, getFieldType(docValueType));
+    if (!INTS.contains(docValueType)) {
+      throw new IllegalArgumentException("docValueType must be one of: " + INTS +"; got " + docValueType);
     }
-    doubleValue = value;
+    fieldsData = Long.valueOf(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)}.
-   */
-  public void setFloat(double value) {
-    if (type == null) {
-      type = DocValues.Type.FLOAT_64;
+  public DocValuesField(String name, float value, DocValues.Type docValueType) {
+    super(name, getFieldType(docValueType));
+    if (docValueType != DocValues.Type.FLOAT_32 &&
+        docValueType != DocValues.Type.FLOAT_64) {
+      throw new IllegalArgumentException("docValueType must be FLOAT_32/64; got " + docValueType);
     }
-    doubleValue = value;
+    fieldsData = Float.valueOf(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.
-   */
-  public void setBytes(BytesRef value, DocValues.Type type) {
-    setBytes(value, type, null);
-  }
-
-  /**
-   * Sets the given {@link BytesRef} value, the field's {@link Type} and the
-   * field's comparator. If the {@link Comparator} is set to <code>null</code>
-   * the default for the given {@link Type} is used instead.
-   * 
-   * @throws IllegalArgumentException
-   *           if the value or the type are null
-   */
-  public void setBytes(BytesRef value, DocValues.Type type, Comparator<BytesRef> comp) {
-    if (value == null) {
-      throw new IllegalArgumentException("value must not be null");
-    }
-    setDocValuesType(type);
-    if (bytes == null) {
-      bytes = BytesRef.deepCopyOf(value);
-    } else {
-      bytes.copyBytes(value);
-    }
-    bytesComparator = comp;
-  }
-
-  /**
-   * Returns the set {@link BytesRef} or <code>null</code> if not set.
-   */
-  public BytesRef getBytes() {
-    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;
-  }
-
-  /**
-   * 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>
-   */
-  public Reader readerValue() {
-    return null;
-  }
-
-  @Override
-  public DocValues.Type docValueType() {
-    return type;
-  }
-
-  @Override
-  public String toString() {
-    final String value;
-    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:
-      // don't use to unicode string this is not necessarily unicode here
-      value = "bytes: " + bytes.toString();
-      break;
-    case FIXED_INTS_16:
-      value = "int16: " + longValue;
-      break;
-    case FIXED_INTS_32:
-      value = "int32: " + longValue;
-      break;
-    case FIXED_INTS_64:
-      value = "int64: " + longValue;
-      break;
-    case FIXED_INTS_8:
-      value = "int8: " + longValue;
-      break;
-    case VAR_INTS:
-      value = "vint: " + longValue;
-      break;
-    case FLOAT_32:
-      value = "float32: " + doubleValue;
-      break;
-    case FLOAT_64:
-      value = "float64: " + doubleValue;
-      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);
+  public DocValuesField(String name, double value, DocValues.Type docValueType) {
+    super(name, getFieldType(docValueType));
+    if (docValueType != DocValues.Type.FLOAT_32 &&
+        docValueType != DocValues.Type.FLOAT_64) {
+      throw new IllegalArgumentException("docValueType must be FLOAT_32/64; got " + docValueType);
     }
-    return valField;
+    fieldsData = Double.valueOf(value);
   }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java Wed Jan 18 22:28:07 2012
@@ -57,15 +57,13 @@ public class DocumentStoredFieldVisitor 
 
   @Override
   public void binaryField(FieldInfo fieldInfo, byte[] value, int offset, int length) throws IOException {
-    doc.add(new BinaryField(fieldInfo.name, value));
+    doc.add(new StoredField(fieldInfo.name, value));
   }
 
   @Override
   public void stringField(FieldInfo fieldInfo, String value) throws IOException {
     final FieldType ft = new FieldType(TextField.TYPE_STORED);
     ft.setStoreTermVectors(fieldInfo.storeTermVector);
-    ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
-    ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
     ft.setStoreTermVectors(fieldInfo.storeTermVector);
     ft.setIndexed(fieldInfo.isIndexed);
     ft.setOmitNorms(fieldInfo.omitNorms);
@@ -75,30 +73,22 @@ public class DocumentStoredFieldVisitor 
 
   @Override
   public void intField(FieldInfo fieldInfo, int value) {
-    FieldType ft = new FieldType(NumericField.TYPE_STORED);
-    ft.setIndexed(fieldInfo.isIndexed);
-    doc.add(new NumericField(fieldInfo.name, ft).setIntValue(value));
+    doc.add(new StoredField(fieldInfo.name, value));
   }
 
   @Override
   public void longField(FieldInfo fieldInfo, long value) {
-    FieldType ft = new FieldType(NumericField.TYPE_STORED);
-    ft.setIndexed(fieldInfo.isIndexed);
-    doc.add(new NumericField(fieldInfo.name, ft).setLongValue(value));
+    doc.add(new StoredField(fieldInfo.name, value));
   }
 
   @Override
   public void floatField(FieldInfo fieldInfo, float value) {
-    FieldType ft = new FieldType(NumericField.TYPE_STORED);
-    ft.setIndexed(fieldInfo.isIndexed);
-    doc.add(new NumericField(fieldInfo.name, ft).setFloatValue(value));
+    doc.add(new StoredField(fieldInfo.name, value));
   }
 
   @Override
   public void doubleField(FieldInfo fieldInfo, double value) {
-    FieldType ft = new FieldType(NumericField.TYPE_STORED);
-    ft.setIndexed(fieldInfo.isIndexed);
-    doc.add(new NumericField(fieldInfo.name, ft).setDoubleValue(value));
+    doc.add(new StoredField(fieldInfo.name, value));
   }
 
   @Override

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/Field.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/Field.java Wed Jan 18 22:28:07 2012
@@ -22,69 +22,103 @@ import java.io.Reader;
 import java.io.StringReader;
 
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.NumericTokenStream;
 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.IndexWriter; // javadocs
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.DocValue;
+import org.apache.lucene.index.IndexableFieldType;
+import org.apache.lucene.index.Norm;
 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
+ * Expert: directly creata a field for a document.  Most
+ * users should use one of the sugar subclasses: {@link
+ * NumericField}, {@link DocValuesField}, {@link
+ * StringField}, {@link TextField}, {@link StoredField}.
+ *
+ * <p/> 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";
-  // the data object for all different kind of field values
+
+  protected final FieldType type;
+  protected final String name;
+
+  // Field's value:
   protected Object fieldsData;
-  // pre-analyzed tokenStream for indexed fields
+
+  // Pre-analyzed tokenStream for indexed fields; this is
+  // separate from fieldsData because you are allowed to
+  // have both; eg maybe field has a String value but you
+  // customize how it's tokenized:
   protected TokenStream tokenStream;
-  // length/offset for all primitive types
-  protected DocValue docValue;
-  
+
+  protected transient NumericTokenStream numericTokenStream;
+
   protected float boost = 1.0f;
 
-  public Field(String name, IndexableFieldType type) {
+  protected Field(String name, FieldType type) {
+    if (name == null) {
+      throw new IllegalArgumentException("name cannot be null");
+    }
     this.name = name;
+    if (type == null) {
+      throw new IllegalArgumentException("type cannot be null");
+    }
     this.type = type;
   }
-  
-  public Field(String name, Reader reader, IndexableFieldType type) {
+
+  /**
+   * Create field with Reader value.
+   */
+  public Field(String name, Reader reader, FieldType type) {
     if (name == null) {
-      throw new NullPointerException("name cannot be null");
+      throw new IllegalArgumentException("name cannot be null");
+    }
+    if (type == null) {
+      throw new IllegalArgumentException("type cannot be null");
     }
     if (reader == null) {
       throw new NullPointerException("reader cannot be null");
     }
+    if (type.stored()) {
+      throw new IllegalArgumentException("fields with a Reader value cannot be stored");
+    }
     if (type.indexed() && !type.tokenized()) {
-      throw new IllegalArgumentException("Non-tokenized fields must use String values");
+      throw new IllegalArgumentException("non-tokenized fields must use String values");
     }
     
     this.name = name;
     this.fieldsData = reader;
     this.type = type;
   }
-  
-  public Field(String name, TokenStream tokenStream, IndexableFieldType type) {
+
+  /**
+   * Create field with TokenStream value.
+   */
+  public Field(String name, TokenStream tokenStream, FieldType type) {
     if (name == null) {
-      throw new NullPointerException("name cannot be null");
+      throw new IllegalArgumentException("name cannot be null");
     }
     if (tokenStream == null) {
       throw new NullPointerException("tokenStream cannot be null");
     }
-    if (type.indexed() && !type.tokenized()) {
-      throw new IllegalArgumentException("Non-tokenized fields must use String values");
+    if (!type.indexed() || !type.tokenized()) {
+      throw new IllegalArgumentException("TokenStream fields must be indexed and tokenized");
+    }
+    if (type.stored()) {
+      throw new IllegalArgumentException("TokenStream fields cannot be stored");
     }
     
     this.name = name;
@@ -93,25 +127,42 @@ public class Field implements IndexableF
     this.type = type;
   }
   
-  public Field(String name, byte[] value, IndexableFieldType type) {
+  /**
+   * Create field with binary value.
+   */
+  public Field(String name, byte[] value, FieldType type) {
     this(name, value, 0, value.length, type);
   }
 
-  public Field(String name, byte[] value, int offset, int length, IndexableFieldType type) {
+  /**
+   * Create field with binary value.
+   */
+  public Field(String name, byte[] value, int offset, int length, FieldType type) {
     this(name, new BytesRef(value, offset, length), type);
   }
 
-  public Field(String name, BytesRef bytes, IndexableFieldType type) {
-    if (type.indexed() && !type.tokenized()) {
-      throw new IllegalArgumentException("Non-tokenized fields must use String values");
+  /**
+   * Create field with binary value.
+   *
+   * <p>NOTE: the provided BytesRef is not copied so be sure
+   * not to change it until you're done with this field.
+   */
+  public Field(String name, BytesRef bytes, FieldType type) {
+    if (name == null) {
+      throw new IllegalArgumentException("name cannot be null");
+    }
+    if (type.indexed()) {
+      throw new IllegalArgumentException("Fields with BytesRef values cannot be indexed");
     }
-
     this.fieldsData = bytes;
     this.type = type;
     this.name = name;
   }
-  
-  public Field(String name, String value, IndexableFieldType type) {
+
+  /**
+   * Create field with String value.
+   */
+  public Field(String name, String value, FieldType type) {
     if (name == null) {
       throw new IllegalArgumentException("name cannot be null");
     }
@@ -122,7 +173,7 @@ public class Field implements IndexableF
       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");
     }
@@ -133,6 +184,54 @@ public class Field implements IndexableF
   }
 
   /**
+   * Create field with an int value.
+   */
+  public Field(String name, int value, FieldType type) {
+    if (name == null) {
+      throw new IllegalArgumentException("name cannot be null");
+    }
+    this.type = type;
+    this.name = name;
+    this.fieldsData = Integer.valueOf(value);
+  }
+
+  /**
+   * Create field with an long value.
+   */
+  public Field(String name, long value, FieldType type) {
+    if (name == null) {
+      throw new IllegalArgumentException("name cannot be null");
+    }
+    this.type = type;
+    this.name = name;
+    this.fieldsData = Long.valueOf(value);
+  }
+
+  /**
+   * Create field with a float value.
+   */
+  public Field(String name, float value, FieldType type) {
+    if (name == null) {
+      throw new IllegalArgumentException("name cannot be null");
+    }
+    this.type = type;
+    this.name = name;
+    this.fieldsData = Float.valueOf(value);
+  }
+
+  /**
+   * Create field with a double value.
+   */
+  public Field(String name, double value, FieldType type) {
+    if (name == null) {
+      throw new IllegalArgumentException("name cannot be null");
+    }
+    this.type = type;
+    this.name = name;
+    this.fieldsData = Double.valueOf(value);
+  }
+
+  /**
    * 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
    * getBinaryValue() must be set.
@@ -175,9 +274,8 @@ public class Field implements IndexableF
    * </p>
    */
   public void setValue(String value) {
-    if (isBinary()) {
-      throw new IllegalArgumentException(
-          "cannot set a String value on a binary field");
+    if (!(fieldsData instanceof String)) {
+      throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to String");
     }
     fieldsData = value;
   }
@@ -187,13 +285,8 @@ public class Field implements IndexableF
    * href="#setValue(java.lang.String)">setValue(String)</a>.
    */
   public void setValue(Reader value) {
-    if (isBinary()) {
-      throw new IllegalArgumentException(
-          "cannot set a Reader value on a binary field");
-    }
-    if (type.stored()) {
-      throw new IllegalArgumentException(
-          "cannot set a Reader value on a stored field");
+    if (!(fieldsData instanceof Reader)) {
+      throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Reader");
     }
     fieldsData = value;
   }
@@ -203,13 +296,66 @@ public class Field implements IndexableF
    * href="#setValue(java.lang.String)">setValue(String)</a>.
    */
   public void setValue(byte[] value) {
-    if (!isBinary()) {
-      throw new IllegalArgumentException(
-          "cannot set a byte[] value on a non-binary field");
+    setValue(new BytesRef(value));
+  }
+
+  /**
+   * Expert: change the value of this field. See <a
+   * href="#setValue(java.lang.String)">setValue(String)</a>.
+   *
+   * <p>NOTE: the provided BytesRef is not copied so be sure
+   * not to change it until you're done with this field.
+   */
+  public void setValue(BytesRef value) {
+    if (!(fieldsData instanceof BytesRef)) {
+      throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to BytesRef");
+    }
+    if (type.indexed()) {
+      throw new IllegalArgumentException("cannot set a Reader value on an indexed field");
     }
-    fieldsData = new BytesRef(value);
+    fieldsData = value;
   }
-  
+
+  public void setValue(int value) {
+    if (!(fieldsData instanceof Integer)) {
+      throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Integer");
+    }
+    if (numericTokenStream != null) {
+      numericTokenStream.setIntValue(value);
+    }
+    fieldsData = Integer.valueOf(value);
+  }
+
+  public void setValue(long value) {
+    if (!(fieldsData instanceof Long)) {
+      throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Long");
+    }
+    if (numericTokenStream != null) {
+      numericTokenStream.setLongValue(value);
+    }
+    fieldsData = Long.valueOf(value);
+  }
+
+  public void setValue(float value) {
+    if (!(fieldsData instanceof Float)) {
+      throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Float");
+    }
+    if (numericTokenStream != null) {
+      numericTokenStream.setFloatValue(value);
+    }
+    fieldsData = Float.valueOf(value);
+  }
+
+  public void setValue(double value) {
+    if (!(fieldsData instanceof Double)) {
+      throw new IllegalArgumentException("cannot change value type from " + fieldsData.getClass().getSimpleName() + " to Double");
+    }
+    if (numericTokenStream != null) {
+      numericTokenStream.setDoubleValue(value);
+    }
+    fieldsData = Double.valueOf(value);
+  }
+
   /**
    * Expert: sets the token stream to be used for indexing and causes
    * isIndexed() and isTokenized() to return true. May be combined with stored
@@ -217,8 +363,10 @@ public class Field implements IndexableF
    */
   public void setTokenStream(TokenStream tokenStream) {
     if (!type.indexed() || !type.tokenized()) {
-      throw new IllegalArgumentException(
-          "cannot set token stream on non indexed and tokenized field");
+      throw new IllegalArgumentException("TokenStream fields must be indexed and tokenized");
+    }
+    if (type.numericType() != null) {
+      throw new IllegalArgumentException("cannot set private TokenStream on numeric fields");
     }
     this.tokenStream = tokenStream;
   }
@@ -236,45 +384,35 @@ public class Field implements IndexableF
    * document.
    *
    * <p>The boost is used to compute the norm factor for the field.  By
-   * default, in the {@link org.apache.lucene.search.similarities.Similarity#computeNorm(FieldInvertState)} method, 
+   * default, in the {@link org.apache.lucene.search.similarities.Similarity#computeNorm(FieldInvertState, Norm)} method, 
    * the boost value is multiplied by the length normalization factor and then
    * rounded by {@link org.apache.lucene.search.similarities.DefaultSimilarity#encodeNormValue(float)} before it is stored in the
    * index.  One should attempt to ensure that this product does not overflow
    * the range of that encoding.
    *
-   * @see org.apache.lucene.search.similarities.Similarity#computeNorm(FieldInvertState)
+   * @see org.apache.lucene.search.similarities.Similarity#computeNorm(FieldInvertState, Norm)
    * @see org.apache.lucene.search.similarities.DefaultSimilarity#encodeNormValue(float)
    */
   public void setBoost(float boost) {
     this.boost = boost;
   }
-  
-  public boolean numeric() {
-    return false;
-  }
 
   public Number numericValue() {
-    return null;
+    if (fieldsData instanceof Number) {
+      return (Number) fieldsData;
+    } else {
+      return null;
+    }
   }
 
-  public NumericField.DataType numericDataType() {
-    return null;
-  }
-  
   public BytesRef binaryValue() {
-    if (!isBinary()) {
-      return null;
-    } else {
+    if (fieldsData instanceof BytesRef) {
       return (BytesRef) fieldsData;
+    } else {
+      return null;
     }
   }
   
-  /** methods from inner IndexableFieldType */
-  
-  public boolean isBinary() {
-    return fieldsData instanceof BytesRef;
-  }
-  
   /** Prints a Field for human consumption. */
   @Override
   public String toString() {
@@ -292,22 +430,8 @@ 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() {
+  /** Returns the {@link FieldType} for this field. */
+  public FieldType fieldType() {
     return type;
   }
 
@@ -319,6 +443,38 @@ public class Field implements IndexableF
       return null;
     }
 
+    final NumericField.DataType numericType = fieldType().numericType();
+    if (numericType != null) {
+      if (numericTokenStream == null) {
+        // lazy init the TokenStream as it is heavy to instantiate
+        // (attributes,...) if not needed (stored field loading)
+        numericTokenStream = new NumericTokenStream(type.numericPrecisionStep());
+        // initialize value in TokenStream
+        final Number val = (Number) fieldsData;
+        switch (numericType) {
+        case INT:
+          numericTokenStream.setIntValue(val.intValue());
+          break;
+        case LONG:
+          numericTokenStream.setLongValue(val.longValue());
+          break;
+        case FLOAT:
+          numericTokenStream.setFloatValue(val.floatValue());
+          break;
+        case DOUBLE:
+          numericTokenStream.setDoubleValue(val.doubleValue());
+          break;
+        default:
+          assert false : "Should never get here";
+        }
+      } else {
+        // OK -- previously cached and we already updated if
+        // setters were called.
+      }
+
+      return numericTokenStream;
+    }
+
     if (!fieldType().tokenized()) {
       if (stringValue() == null) {
         throw new IllegalArgumentException("Non-Tokenized Fields must have a String value");
@@ -355,6 +511,449 @@ public class Field implements IndexableF
       return analyzer.tokenStream(name(), new StringReader(stringValue()));
     }
 
-    throw new IllegalArgumentException("Field must have either TokenStream, String or Reader value");
+    throw new IllegalArgumentException("Field must have either TokenStream, String, Reader or Number value");
+  }
+
+  
+  //
+  // Deprecated transition API below:
+  //
+
+  /** Specifies whether and how a field should be stored.
+   *
+   *  @deprecated This is here only to ease transition from
+   *  the pre-4.0 APIs. */
+  @Deprecated
+  public static enum Store {
+
+    /** Store the original field value in the index. This is useful for short texts
+     * like a document's title which should be displayed with the results. The
+     * value is stored in its original form, i.e. no analyzer is used before it is
+     * stored.
+     */
+    YES {
+      @Override
+      public boolean isStored() { return true; }
+    },
+
+    /** Do not store the field value in the index. */
+    NO {
+      @Override
+      public boolean isStored() { return false; }
+    };
+
+    public abstract boolean isStored();
+  }
+
+  /** Specifies whether and how a field should be indexed.
+   *
+   *  @deprecated This is here only to ease transition from
+   *  the pre-4.0 APIs. */
+  @Deprecated
+  public static enum Index {
+
+    /** Do not index the field value. This field can thus not be searched,
+     * but one can still access its contents provided it is
+     * {@link Field.Store stored}. */
+    NO {
+      @Override
+      public boolean isIndexed()  { return false; }
+      @Override
+      public boolean isAnalyzed() { return false; }
+      @Override
+      public boolean omitNorms()  { return true;  }   
+    },
+
+    /** Index the tokens produced by running the field's
+     * value through an Analyzer.  This is useful for
+     * common text. */
+    ANALYZED {
+      @Override
+      public boolean isIndexed()  { return true;  }
+      @Override
+      public boolean isAnalyzed() { return true;  }
+      @Override
+      public boolean omitNorms()  { return false; }   	
+    },
+
+    /** Index the field's value without using an Analyzer, so it can be searched.
+     * As no analyzer is used the value will be stored as a single term. This is
+     * useful for unique Ids like product numbers.
+     */
+    NOT_ANALYZED {
+      @Override
+      public boolean isIndexed()  { return true;  }
+      @Override
+      public boolean isAnalyzed() { return false; }
+      @Override
+      public boolean omitNorms()  { return false; }   	
+    },
+
+    /** Expert: Index the field's value without an Analyzer,
+     * and also disable the indexing of norms.  Note that you
+     * can also separately enable/disable norms by calling
+     * {@link FieldType#setOmitNorms}.  No norms means that
+     * index-time field and document boosting and field
+     * length normalization are disabled.  The benefit is
+     * less memory usage as norms take up one byte of RAM
+     * per indexed field for every document in the index,
+     * during searching.  Note that once you index a given
+     * field <i>with</i> norms enabled, disabling norms will
+     * have no effect.  In other words, for this to have the
+     * above described effect on a field, all instances of
+     * that field must be indexed with NOT_ANALYZED_NO_NORMS
+     * from the beginning. */
+    NOT_ANALYZED_NO_NORMS {
+      @Override
+      public boolean isIndexed()  { return true;  }
+      @Override
+      public boolean isAnalyzed() { return false; }
+      @Override
+      public boolean omitNorms()  { return true;  }   	
+    },
+
+    /** Expert: Index the tokens produced by running the
+     *  field's value through an Analyzer, and also
+     *  separately disable the storing of norms.  See
+     *  {@link #NOT_ANALYZED_NO_NORMS} for what norms are
+     *  and why you may want to disable them. */
+    ANALYZED_NO_NORMS {
+      @Override
+      public boolean isIndexed()  { return true;  }
+      @Override
+      public boolean isAnalyzed() { return true;  }
+      @Override
+      public boolean omitNorms()  { return true;  }   	
+    };
+
+    /** Get the best representation of the index given the flags. */
+    public static Index toIndex(boolean indexed, boolean analyzed) {
+      return toIndex(indexed, analyzed, false);
+    }
+
+    /** Expert: Get the best representation of the index given the flags. */
+    public static Index toIndex(boolean indexed, boolean analyzed, boolean omitNorms) {
+
+      // If it is not indexed nothing else matters
+      if (!indexed) {
+        return Index.NO;
+      }
+
+      // typical, non-expert
+      if (!omitNorms) {
+        if (analyzed) {
+          return Index.ANALYZED;
+        }
+        return Index.NOT_ANALYZED;
+      }
+
+      // Expert: Norms omitted
+      if (analyzed) {
+        return Index.ANALYZED_NO_NORMS;
+      }
+      return Index.NOT_ANALYZED_NO_NORMS;
+    }
+
+    public abstract boolean isIndexed();
+    public abstract boolean isAnalyzed();
+    public abstract boolean omitNorms();  	
+  }
+
+  /** Specifies whether and how a field should have term vectors.
+   *
+   *  @deprecated This is here only to ease transition from
+   *  the pre-4.0 APIs. */
+  @Deprecated
+  public static enum TermVector {
+    
+    /** Do not store term vectors. 
+     */
+    NO {
+      @Override
+      public boolean isStored()      { return false; }
+      @Override
+      public boolean withPositions() { return false; }
+      @Override
+      public boolean withOffsets()   { return false; }
+    },
+    
+    /** Store the term vectors of each document. A term vector is a list
+     * of the document's terms and their number of occurrences in that document. */
+    YES {
+      @Override
+      public boolean isStored()      { return true;  }
+      @Override
+      public boolean withPositions() { return false; }
+      @Override
+      public boolean withOffsets()   { return false; }
+    },
+    
+    /**
+     * Store the term vector + token position information
+     * 
+     * @see #YES
+     */ 
+    WITH_POSITIONS {
+      @Override
+      public boolean isStored()      { return true;  }
+      @Override
+      public boolean withPositions() { return true;  }
+      @Override
+      public boolean withOffsets()   { return false; }
+    },
+    
+    /**
+     * Store the term vector + Token offset information
+     * 
+     * @see #YES
+     */ 
+    WITH_OFFSETS {
+      @Override
+      public boolean isStored()      { return true;  }
+      @Override
+      public boolean withPositions() { return false; }
+      @Override
+      public boolean withOffsets()   { return true;  }
+    },
+    
+    /**
+     * Store the term vector + Token position and offset information
+     * 
+     * @see #YES
+     * @see #WITH_POSITIONS
+     * @see #WITH_OFFSETS
+     */ 
+    WITH_POSITIONS_OFFSETS {
+      @Override
+      public boolean isStored()      { return true;  }
+      @Override
+      public boolean withPositions() { return true;  }
+      @Override
+      public boolean withOffsets()   { return true;  }
+    };
+
+    /** Get the best representation of a TermVector given the flags. */
+    public static TermVector toTermVector(boolean stored, boolean withOffsets, boolean withPositions) {
+
+      // If it is not stored, nothing else matters.
+      if (!stored) {
+        return TermVector.NO;
+      }
+
+      if (withOffsets) {
+        if (withPositions) {
+          return Field.TermVector.WITH_POSITIONS_OFFSETS;
+        }
+        return Field.TermVector.WITH_OFFSETS;
+      }
+
+      if (withPositions) {
+        return Field.TermVector.WITH_POSITIONS;
+      }
+      return Field.TermVector.YES;
+    }
+
+    public abstract boolean isStored();
+    public abstract boolean withPositions();
+    public abstract boolean withOffsets();
+  }
+
+  /** Translates the pre-4.0 enums for specifying how a
+   *  field should be indexed into the 4.0 {@link FieldType}
+   *  approach.
+   *
+   * @deprecated This is here only to ease transition from
+   * the pre-4.0 APIs.
+   */
+  @Deprecated
+  public static final FieldType translateFieldType(Store store, Index index, TermVector termVector) {
+    final FieldType ft = new FieldType();
+
+    ft.setStored(store == Store.YES);
+
+    switch(index) {
+    case ANALYZED:
+      ft.setIndexed(true);
+      ft.setTokenized(true);
+      break;
+    case ANALYZED_NO_NORMS:
+      ft.setIndexed(true);
+      ft.setTokenized(true);
+      ft.setOmitNorms(true);
+      break;
+    case NOT_ANALYZED:
+      ft.setIndexed(true);
+      break;
+    case NOT_ANALYZED_NO_NORMS:
+      ft.setIndexed(true);
+      ft.setOmitNorms(true);
+      break;
+    case NO:
+      break;
+    }
+
+    switch(termVector) {
+    case NO:
+      break;
+    case YES:
+      ft.setStoreTermVectors(true);
+      break;
+    case WITH_POSITIONS:
+      ft.setStoreTermVectors(true);
+      ft.setStoreTermVectorPositions(true);
+      break;
+    case WITH_OFFSETS:
+      ft.setStoreTermVectors(true);
+      ft.setStoreTermVectorOffsets(true);
+      break;
+    case WITH_POSITIONS_OFFSETS:
+      ft.setStoreTermVectors(true);
+      ft.setStoreTermVectorPositions(true);
+      ft.setStoreTermVectorOffsets(true);
+      break;
+    }
+    ft.freeze();
+    return ft;
+  }
+
+  /**
+   * Create a field by specifying its name, value and how it will
+   * be saved in the index. Term vectors will not be stored in the index.
+   * 
+   * @param name The name of the field
+   * @param value The string to process
+   * @param store Whether <code>value</code> should be stored in the index
+   * @param index Whether the field should be indexed, and if so, if it should
+   *  be tokenized before indexing 
+   * @throws NullPointerException if name or value is <code>null</code>
+   * @throws IllegalArgumentException if the field is neither stored nor indexed 
+   *
+   * @deprecated Use {@link StringField}, {@link TextField} instead. */
+  @Deprecated
+  public Field(String name, String value, Store store, Index index) {
+    this(name, value, translateFieldType(store, index, TermVector.NO));
+  }
+
+  /**
+   * Create a field by specifying its name, value and how it will
+   * be saved in the index.
+   * 
+   * @param name The name of the field
+   * @param value The string to process
+   * @param store Whether <code>value</code> should be stored in the index
+   * @param index Whether the field should be indexed, and if so, if it should
+   *  be tokenized before indexing 
+   * @param termVector Whether term vector should be stored
+   * @throws NullPointerException if name or value is <code>null</code>
+   * @throws IllegalArgumentException in any of the following situations:
+   * <ul> 
+   *  <li>the field is neither stored nor indexed</li> 
+   *  <li>the field is not indexed but termVector is <code>TermVector.YES</code></li>
+   * </ul> 
+   *
+   * @deprecated Use {@link StringField}, {@link TextField} instead. */
+  @Deprecated
+  public Field(String name, String value, Store store, Index index, TermVector termVector) {  
+    this(name, value, translateFieldType(store, index, termVector));
+  }
+
+  /**
+   * Create a tokenized and indexed field that is not stored. Term vectors will
+   * not be stored.  The Reader is read only when the Document is added to the index,
+   * i.e. you may not close the Reader until {@link IndexWriter#addDocument}
+   * has been called.
+   * 
+   * @param name The name of the field
+   * @param reader The reader with the content
+   * @throws NullPointerException if name or reader is <code>null</code>
+   *
+   * @deprecated Use {@link TextField} instead.
+   */
+  @Deprecated
+  public Field(String name, Reader reader) {
+    this(name, reader, TermVector.NO);
+  }
+
+  /**
+   * Create a tokenized and indexed field that is not stored, optionally with 
+   * storing term vectors.  The Reader is read only when the Document is added to the index,
+   * i.e. you may not close the Reader until {@link IndexWriter#addDocument}
+   * has been called.
+   * 
+   * @param name The name of the field
+   * @param reader The reader with the content
+   * @param termVector Whether term vector should be stored
+   * @throws NullPointerException if name or reader is <code>null</code>
+   *
+   * @deprecated Use {@link TextField} instead.
+   */ 
+  @Deprecated
+  public Field(String name, Reader reader, TermVector termVector) {
+    this(name, reader, translateFieldType(Store.NO, Index.ANALYZED, termVector));
+  }
+
+  /**
+   * Create a tokenized and indexed field that is not stored. Term vectors will
+   * not be stored. This is useful for pre-analyzed fields.
+   * The TokenStream is read only when the Document is added to the index,
+   * i.e. you may not close the TokenStream until {@link IndexWriter#addDocument}
+   * has been called.
+   * 
+   * @param name The name of the field
+   * @param tokenStream The TokenStream with the content
+   * @throws NullPointerException if name or tokenStream is <code>null</code>
+   *
+   * @deprecated Use {@link TextField} instead
+   */ 
+  @Deprecated
+  public Field(String name, TokenStream tokenStream) {
+    this(name, tokenStream, TermVector.NO);
+  }
+
+  /**
+   * Create a tokenized and indexed field that is not stored, optionally with 
+   * storing term vectors.  This is useful for pre-analyzed fields.
+   * The TokenStream is read only when the Document is added to the index,
+   * i.e. you may not close the TokenStream until {@link IndexWriter#addDocument}
+   * has been called.
+   * 
+   * @param name The name of the field
+   * @param tokenStream The TokenStream with the content
+   * @param termVector Whether term vector should be stored
+   * @throws NullPointerException if name or tokenStream is <code>null</code>
+   *
+   * @deprecated Use {@link TextField} instead
+   */ 
+  @Deprecated
+  public Field(String name, TokenStream tokenStream, TermVector termVector) {
+    this(name, tokenStream, translateFieldType(Store.NO, Index.ANALYZED, termVector));
+  }
+
+  /**
+   * Create a stored field with binary value. Optionally the value may be compressed.
+   * 
+   * @param name The name of the field
+   * @param value The binary value
+   *
+   * @deprecated Use {@link StoredField} instead.
+   */
+  @Deprecated
+  public Field(String name, byte[] value) {
+    this(name, value, translateFieldType(Store.YES, Index.NO, TermVector.NO));
+  }
+
+  /**
+   * Create a stored field with binary value. Optionally the value may be compressed.
+   * 
+   * @param name The name of the field
+   * @param value The binary value
+   * @param offset Starting offset in value where this Field's bytes are
+   * @param length Number of bytes to use for this Field, starting at offset
+   *
+   * @deprecated Use {@link StoredField} instead.
+   */
+  @Deprecated
+  public Field(String name, byte[] value, int offset, int length) {
+    this(name, value, offset, length, translateFieldType(Store.YES, Index.NO, TermVector.NO));
   }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/FieldType.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/FieldType.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/FieldType.java Wed Jan 18 22:28:07 2012
@@ -17,8 +17,11 @@ 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;
+import org.apache.lucene.search.NumericRangeQuery; // javadocs
+import org.apache.lucene.util.NumericUtils;
 
 public class FieldType implements IndexableFieldType {
 
@@ -30,9 +33,12 @@ 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 NumericField.DataType numericType;
   private boolean frozen;
+  private int numericPrecisionStep = NumericUtils.PRECISION_STEP_DEFAULT;
 
-  public FieldType(IndexableFieldType ref) {
+  public FieldType(FieldType ref) {
     this.indexed = ref.indexed();
     this.stored = ref.stored();
     this.tokenized = ref.tokenized();
@@ -41,6 +47,8 @@ public class FieldType implements Indexa
     this.storeTermVectorPositions = ref.storeTermVectorPositions();
     this.omitNorms = ref.omitNorms();
     this.indexOptions = ref.indexOptions();
+    this.docValueType = ref.docValueType();
+    this.numericType = ref.numericType();
     // Do not copy frozen!
   }
   
@@ -49,7 +57,7 @@ public class FieldType implements Indexa
 
   private void checkIfFrozen() {
     if (frozen) {
-      throw new IllegalStateException();
+      throw new IllegalStateException("this FieldType is already frozen and cannot be changed");
     }
   }
 
@@ -134,6 +142,42 @@ public class FieldType implements Indexa
     this.indexOptions = value;
   }
 
+  public void setDocValueType(DocValues.Type type) {
+    checkIfFrozen();
+    docValueType = type;
+  }
+  
+  @Override
+  public DocValues.Type docValueType() {
+    return docValueType;
+  }
+
+  public void setNumericType(NumericField.DataType type) {
+    checkIfFrozen();
+    numericType = type;
+  }
+
+  /** Numeric {@link NumericField.DataType}; if
+   *  non-null then the field's value will be indexed
+   *  numerically so that {@link NumericRangeQuery} can be
+   *  used at search time. */
+  public NumericField.DataType numericType() {
+    return numericType;
+  }
+
+  public void setNumericPrecisionStep(int precisionStep) {
+    checkIfFrozen();
+    if (precisionStep < 1) {
+      throw new IllegalArgumentException("precisionStep must be >= 1 (got " + precisionStep + ")");
+    }
+    this.numericPrecisionStep = precisionStep;
+  }
+
+  /** Precision step for numeric field. */
+  public int numericPrecisionStep() {
+    return numericPrecisionStep;
+  }
+
   /** Prints a Field for human consumption. */
   @Override
   public final String toString() {
@@ -172,6 +216,16 @@ public class FieldType implements Indexa
         result.append(",indexOptions=");
         result.append(indexOptions);
       }
+      if (numericType != null) {
+        result.append(",numericType=");
+        result.append(numericType);
+        result.append(",numericPrecisionStep=");
+        result.append(numericPrecisionStep);
+      }
+    }
+    if (docValueType != null) {
+      result.append(",docValueType=");
+      result.append(docValueType);
     }
     
     return result.toString();

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/NumericField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/NumericField.java?rev=1233096&r1=1233095&r2=1233096&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/NumericField.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/document/NumericField.java Wed Jan 18 22:28:07 2012
@@ -17,17 +17,14 @@ package org.apache.lucene.document;
  * limitations under the License.
  */
 
-import java.io.Reader;
 
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.NumericTokenStream;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.analysis.NumericTokenStream; // javadocs
 import org.apache.lucene.document.NumericField.DataType;
-import org.apache.lucene.util.NumericUtils;
-import org.apache.lucene.search.NumericRangeQuery; // javadocs
-import org.apache.lucene.search.NumericRangeFilter; // javadocs
+import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.search.FieldCache; // javadocs
+import org.apache.lucene.search.NumericRangeFilter; // javadocs
+import org.apache.lucene.search.NumericRangeQuery; // javadocs
+import org.apache.lucene.util.NumericUtils;
 
 /**
  * <p>
@@ -36,20 +33,20 @@ import org.apache.lucene.search.FieldCac
  * int value:
  * 
  * <pre>
- * document.add(new NumericField(name).setIntValue(value));
+ * document.add(new NumericField(name, value));
  * </pre>
  * 
  * For optimal performance, re-use the <code>NumericField</code> and
  * {@link Document} instance for more than one document:
  * 
  * <pre>
- *  NumericField field = new NumericField(name);
+ *  NumericField field = new NumericField(name, NumericField.DataType.INT);
  *  Document document = new Document();
  *  document.add(field);
  * 
  *  for(all documents) {
  *    ...
- *    field.setIntValue(value)
+ *    field.setValue(value)
  *    writer.addDocument(document);
  *    ...
  *  }
@@ -77,8 +74,8 @@ import org.apache.lucene.search.FieldCac
  *
  * <p>By default, a <code>NumericField</code>'s value is not stored but
  * is indexed for range filtering and sorting.  You can use
- * the {@link #NumericField(String, FieldType)}
- * constructor if you need to change these defaults.</p>
+ * {@link Field#Field(String,int,FieldType)}, etc.,
+ * if you need to change these defaults.</p>
  *
  * <p>You may add the same field name as a <code>NumericField</code> to
  * the same document more than once.  Range querying and
@@ -104,8 +101,8 @@ import org.apache.lucene.search.FieldCac
  * but may result in faster range search performance.  The
  * default value, 4, was selected for a reasonable tradeoff
  * of disk space consumption versus performance.  You can
- * use the expert constructor {@link
- * #NumericField(String,int, FieldType)} if you'd
+ * create a custom {@link FieldType} and invoke the {@link
+ * FieldType#setNumericPrecisionStep} method if you'd
  * like to change the value.  Note that you must also
  * specify a congruent value when creating {@link
  * NumericRangeQuery} or {@link NumericRangeFilter}.
@@ -137,244 +134,90 @@ public final class NumericField extends 
   /** Data type of the value in {@link NumericField}.
    * @since 3.2
    */
-  public static enum DataType { INT, LONG, FLOAT, DOUBLE }
+  public static enum DataType {INT, LONG, FLOAT, DOUBLE}
 
-  public static final FieldType TYPE_UNSTORED = new FieldType();
-  public static final FieldType TYPE_STORED = new FieldType();
-  static {
-    TYPE_UNSTORED.setIndexed(true);
-    TYPE_UNSTORED.setTokenized(true);
-    TYPE_UNSTORED.setOmitNorms(true);
-    TYPE_UNSTORED.setIndexOptions(IndexOptions.DOCS_ONLY);
-    TYPE_UNSTORED.freeze();
-
-    TYPE_STORED.setIndexed(true);
-    TYPE_STORED.setStored(true);
-    TYPE_STORED.setTokenized(true);
-    TYPE_STORED.setOmitNorms(true);
-    TYPE_STORED.setIndexOptions(IndexOptions.DOCS_ONLY);
-    TYPE_STORED.freeze();
+  /** @lucene.experimental */
+  public static FieldType getFieldType(DataType type, boolean stored) {
+    final FieldType ft = new FieldType();
+    ft.setIndexed(true);
+    ft.setStored(stored);
+    ft.setTokenized(true);
+    ft.setOmitNorms(true);
+    ft.setIndexOptions(IndexOptions.DOCS_ONLY);
+    ft.setNumericType(type);
+    ft.freeze();
+    return ft;
+  }
+
+  private static final FieldType INT_TYPE = getFieldType(DataType.INT, false);
+  private static final FieldType LONG_TYPE = getFieldType(DataType.LONG, false);
+  private static final FieldType FLOAT_TYPE = getFieldType(DataType.FLOAT, false);
+  private static final FieldType DOUBLE_TYPE = getFieldType(DataType.DOUBLE, false);
+
+  /** Creates an int NumericField with the provided value
+   *  and default <code>precisionStep</code> {@link
+   *  NumericUtils#PRECISION_STEP_DEFAULT} (4). */
+  public NumericField(String name, int value) {
+    super(name, INT_TYPE);
+    fieldsData = Integer.valueOf(value);
   }
 
-  //public static enum DataType { INT, LONG, FLOAT, DOUBLE }
-  
-  private DataType dataType;
-  private transient NumericTokenStream numericTS;
-  private final int precisionStep;
-  
-  /**
-   * Creates a field for numeric values using the default
-   * <code>precisionStep</code> {@link NumericUtils#PRECISION_STEP_DEFAULT} (4).
-   * The instance is not yet initialized with a numeric value, before indexing a
-   * document containing this field, set a value using the various set
-   * <em>???</em>Value() methods. This constructor creates an indexed, but not
-   * stored field.
-   * 
-   * @param name
-   *          the field name
-   */
-  public NumericField(String name) {
-    this(name, NumericUtils.PRECISION_STEP_DEFAULT, NumericField.TYPE_UNSTORED);
+  /** Creates a long NumericField with the provided value.
+   *  and default <code>precisionStep</code> {@link
+   *  NumericUtils#PRECISION_STEP_DEFAULT} (4). */
+  public NumericField(String name, long value) {
+    super(name, LONG_TYPE);
+    fieldsData = Long.valueOf(value);
   }
-  
-  /**
-   * Creates a field for numeric values using the default
-   * <code>precisionStep</code> {@link NumericUtils#PRECISION_STEP_DEFAULT} (4).
-   * The instance is not yet initialized with a numeric value, before indexing a
-   * document containing this field, set a value using the various set
-   * <em>???</em>Value() methods.
-   * 
-   * @param name
-   *          the field name
-   * @param type
-   *          if the defualt field should be altered, e.g. stored, 
-   *          {@link Document#getField} then returns {@code NumericField} 
-   *          instances on search results, or indexed using 
-   *          {@link NumericTokenStream}
-   */
-  public NumericField(String name, FieldType type) {
-    this(name, NumericUtils.PRECISION_STEP_DEFAULT, type);
+
+  /** Creates a float NumericField with the provided value.
+   *  and default <code>precisionStep</code> {@link
+   *  NumericUtils#PRECISION_STEP_DEFAULT} (4). */
+  public NumericField(String name, float value) {
+    super(name, FLOAT_TYPE);
+    fieldsData = Float.valueOf(value);
   }
-  
-  /**
-   * Creates a field for numeric values with the specified
-   * <code>precisionStep</code>. The instance is not yet initialized with a
-   * numeric value, before indexing a document containing this field, set a
-   * value using the various set<em>???</em>Value() methods. This constructor
-   * creates an indexed, but not stored field.
-   * 
-   * @param name
-   *          the field name
-   * @param precisionStep
-   *          the used <a
-   *          href="../search/NumericRangeQuery.html#precisionStepDesc"
-   *          >precision step</a>
-   */
-  public NumericField(String name, int precisionStep) {
-    this(name, precisionStep, NumericField.TYPE_UNSTORED);
+
+  /** Creates a double NumericField with the provided value.
+   *  and default <code>precisionStep</code> {@link
+   *  NumericUtils#PRECISION_STEP_DEFAULT} (4). */
+  public NumericField(String name, double value) {
+    super(name, DOUBLE_TYPE);
+    fieldsData = Double.valueOf(value);
   }
   
-  /**
-   * Creates a field for numeric values with the specified
-   * <code>precisionStep</code>. The instance is not yet initialized with a
-   * numeric value, before indexing a document containing this field, set a
-   * value using the various set<em>???</em>Value() methods.
-   * 
-   * @param name
-   *          the field name
-   * @param precisionStep
-   *          the used <a
-   *          href="../search/NumericRangeQuery.html#precisionStepDesc"
-   *          >precision step</a>
-   * @param type
-   *          if the defualt field should be altered, e.g. stored, 
-   *          {@link Document#getField} then returns {@code NumericField} 
-   *          instances on search results, or indexed using 
-   *          {@link NumericTokenStream}
-   */
-  public NumericField(String name, int precisionStep, FieldType type) {
+  public NumericField(String name, Number value, FieldType type) {
     super(name, type);
-    if (precisionStep < 1)
-      throw new IllegalArgumentException("precisionStep must be >=1");
-    this.precisionStep = precisionStep;
-  }
-  
-  /** Returns a {@link NumericTokenStream} for indexing the numeric value. */
-  public TokenStream tokenStream(Analyzer analyzer) {
-    if (!type.indexed()) return null;
-    if (numericTS == null) {
-      // lazy init the TokenStream as it is heavy to instantiate
-      // (attributes,...),
-      // if not needed (stored field loading)
-      numericTS = new NumericTokenStream(precisionStep);
-      // initialize value in TokenStream
-      if (fieldsData != null) {
-        assert dataType != null;
-        final Number val = (Number) fieldsData;
-        switch (dataType) {
-          case INT:
-            numericTS.setIntValue(val.intValue());
-            break;
-          case LONG:
-            numericTS.setLongValue(val.longValue());
-            break;
-          case FLOAT:
-            numericTS.setFloatValue(val.floatValue());
-            break;
-          case DOUBLE:
-            numericTS.setDoubleValue(val.doubleValue());
-            break;
-          default:
-            assert false : "Should never get here";
-        }
-      }
+    final NumericField.DataType numericType = type.numericType();
+    if (numericType == null) {
+      throw new IllegalArgumentException("FieldType.numericType() cannot be null");
     }
-    return numericTS;
-  }
-  
-  /** Returns always <code>null</code> for numeric fields */
-  public Reader readerValue() {
-    return null;
-  }
-  
-  /**
-   * Returns the numeric value as a string. It is recommended to
-   * use {@link Document#getField} instead that returns {@code NumericField}
-   * instances. You can then use {@link #numericValue} to return the stored
-   * value.
-   */
-  @Override
-  public String stringValue() {
-    return (fieldsData == null) ? null : fieldsData.toString();
-  }
-  
-  /**
-   * Returns the current numeric value as a subclass of {@link Number},
-   * <code>null</code> if not yet initialized.
-   */
-  @Override
-  public Number numericValue() {
-    return (Number) fieldsData;
-  }
-  
-  /** Returns the precision step. */
-  public int getPrecisionStep() {
-    return precisionStep;
-  }
-  
-  /**
-   * Returns the data type of the current value, {@code null} if not yet set.
-   * 
-   * @since 3.2
-   */
-  @Override
-  public DataType numericDataType() {
-    return dataType;
-  }
 
-  @Override
-  public boolean numeric() {
-    return true;
-  }
+    switch(numericType) {
+    case INT:
+      if (!(value instanceof Integer)) {
+        throw new IllegalArgumentException("value must be an Integer but got " + value);
+      }
+      break;
+    case LONG:
+      if (!(value instanceof Long)) {
+        throw new IllegalArgumentException("value must be a Long but got " + value);
+      }
+      break;
+    case FLOAT:
+      if (!(value instanceof Float)) {
+        throw new IllegalArgumentException("value must be a Float but got " + value);
+      }
+      break;
+    case DOUBLE:
+      if (!(value instanceof Double)) {
+        throw new IllegalArgumentException("value must be a Double but got " + value);
+      }
+      break;
+    default:
+      assert false : "Should never get here";
+    }
 
-  /**
-   * Initializes the field with the supplied <code>long</code> value.
-   * 
-   * @param value
-   *          the numeric value
-   * @return this instance, because of this you can use it the following way:
-   *         <code>document.add(new NumericField(name, precisionStep).setLongValue(value))</code>
-   */
-  public NumericField setLongValue(final long value) {
-    if (numericTS != null) numericTS.setLongValue(value);
-    fieldsData = Long.valueOf(value);
-    dataType = DataType.LONG;
-    return this;
-  }
-  
-  /**
-   * Initializes the field with the supplied <code>int</code> value.
-   * 
-   * @param value
-   *          the numeric value
-   * @return this instance, because of this you can use it the following way:
-   *         <code>document.add(new NumericField(name, precisionStep).setIntValue(value))</code>
-   */
-  public NumericField setIntValue(final int value) {
-    if (numericTS != null) numericTS.setIntValue(value);
-    fieldsData = Integer.valueOf(value);
-    dataType = DataType.INT;
-    return this;
+    fieldsData = value;
   }
-  
-  /**
-   * Initializes the field with the supplied <code>double</code> value.
-   * 
-   * @param value
-   *          the numeric value
-   * @return this instance, because of this you can use it the following way:
-   *         <code>document.add(new NumericField(name, precisionStep).setDoubleValue(value))</code>
-   */
-  public NumericField setDoubleValue(final double value) {
-    if (numericTS != null) numericTS.setDoubleValue(value);
-    fieldsData = Double.valueOf(value);
-    dataType = DataType.DOUBLE;
-    return this;
-  }
-  
-  /**
-   * Initializes the field with the supplied <code>float</code> value.
-   * 
-   * @param value
-   *          the numeric value
-   * @return this instance, because of this you can use it the following way:
-   *         <code>document.add(new NumericField(name, precisionStep).setFloatValue(value))</code>
-   */
-  public NumericField setFloatValue(final float value) {
-    if (numericTS != null) numericTS.setFloatValue(value);
-    fieldsData = Float.valueOf(value);
-    dataType = DataType.FLOAT;
-    return this;
-  }
-  
 }