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 2011/08/27 15:27:12 UTC

svn commit: r1162347 [2/15] - in /lucene/dev/trunk: lucene/ lucene/contrib/ lucene/contrib/demo/src/java/org/apache/lucene/demo/ lucene/contrib/demo/src/java/org/apache/lucene/demo/xmlparser/ lucene/contrib/highlighter/src/java/org/apache/lucene/search...

Modified: lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java (original)
+++ lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java Sat Aug 27 13:27:01 2011
@@ -29,6 +29,8 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.MultiNorms;
@@ -204,19 +206,44 @@ public class TestIndicesEquals extends L
 
 
   private void assembleDocument(Document document, int i) {
-    document.add(new Field("a", i + " Do you really want to go and live in that house all winter?", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+    FieldType customType = new FieldType(TextField.TYPE_STORED);
+    customType.setStoreTermVectors(true);
+    customType.setStoreTermVectorOffsets(true);
+    customType.setStoreTermVectorPositions(true);
+    //document.add(new Field("a", i + " Do you really want to go and live in that house all winter?", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+    document.add(new Field("a", customType, i + " Do you really want to go and live in that house all winter?"));
     if (i > 0) {
-      document.add(new Field("b0", i + " All work and no play makes Jack a dull boy", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
-      document.add(new Field("b1", i + " All work and no play makes Jack a dull boy", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS, Field.TermVector.NO));
-      document.add(new Field("b2", i + " All work and no play makes Jack a dull boy", Field.Store.NO, Field.Index.NOT_ANALYZED, Field.TermVector.NO));
-      document.add(new Field("b3", i + " All work and no play makes Jack a dull boy", Field.Store.YES, Field.Index.NO, Field.TermVector.NO));
+      //document.add(new Field("b0", i + " All work and no play makes Jack a dull boy", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+      document.add(new Field("b0", customType, i + " All work and no play makes Jack a dull boy"));
+
+      //document.add(new Field("b1", i + " All work and no play makes Jack a dull boy", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS, Field.TermVector.NO));
+      FieldType customType2 = new FieldType(TextField.TYPE_STORED);
+      customType2.setTokenized(false);
+      customType2.setOmitNorms(true);
+      document.add(new Field("b1", customType2, i + " All work and no play makes Jack a dull boy"));
+      
+      //document.add(new Field("b2", i + " All work and no play makes Jack a dull boy", Field.Store.NO, Field.Index.NOT_ANALYZED, Field.TermVector.NO));
+      FieldType customType3 = new FieldType(TextField.TYPE_UNSTORED);
+      customType3.setTokenized(false);
+      document.add(new Field("b1", customType3, i + " All work and no play makes Jack a dull boy"));
+      
+      //document.add(new Field("b3", i + " All work and no play makes Jack a dull boy", Field.Store.YES, Field.Index.NO, Field.TermVector.NO));
+      FieldType customType4 = new FieldType(TextField.TYPE_STORED);
+      customType4.setIndexed(false);
+      customType4.setTokenized(false);
+      document.add(new Field("b1", customType4, i + " All work and no play makes Jack a dull boy"));
       if (i > 1) {
-        document.add(new Field("c", i + " Redrum redrum", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+        //document.add(new Field("c", i + " Redrum redrum", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+        document.add(new Field("c", customType, i + " Redrum redrum"));
         if (i > 2) {
-          document.add(new Field("d", i + " Hello Danny, come and play with us... forever and ever. and ever.", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+          //document.add(new Field("d", i + " Hello Danny, come and play with us... forever and ever. and ever.", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+          document.add(new Field("d", customType, i + " Hello Danny, come and play with us... forever and ever. and ever."));
           if (i > 3) {
-            Field f = new Field("e", i + " Heres Johnny!", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
-            f.setOmitNorms(true);
+            //Field f = new Field("e", i + " Heres Johnny!", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
+            //f.setOmitNorms(true);
+            FieldType customType5 = new FieldType(TextField.TYPE_UNSTORED);
+            customType5.setOmitNorms(true);
+            Field f = new Field("e", customType5, i + " Heres Johnny!");
             document.add(f);
             if (i > 4) {
               final List<Token> tokens = new ArrayList<Token>(2);
@@ -247,7 +274,8 @@ public class TestIndicesEquals extends L
                 }
               };
               
-              document.add(new Field("f", ts));
+              //document.add(new Field("f", ts));
+              document.add(new TextField("f", ts));
             }
           }
         }

Modified: lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestRealTime.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestRealTime.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestRealTime.java (original)
+++ lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestRealTime.java Sat Aug 27 13:27:01 2011
@@ -19,7 +19,7 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.util.LuceneTestCase;
@@ -43,7 +43,7 @@ public class TestRealTime extends Lucene
     Collector collector;
 
     doc = new Document();
-    doc.add(new Field("f", "a", Field.Store.NO, Field.Index.NOT_ANALYZED));
+    doc.add(new StringField("f", "a"));
     writer.addDocument(doc);
     writer.commit();
 
@@ -52,7 +52,7 @@ public class TestRealTime extends Lucene
     assertEquals(1, collector.hits);
 
     doc = new Document();
-    doc.add(new Field("f", "a", Field.Store.NO, Field.Index.NOT_ANALYZED));
+    doc.add(new StringField("f", "a"));
     writer.addDocument(doc);
     writer.commit();
 

Modified: lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestUnoptimizedReaderOnConstructor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestUnoptimizedReaderOnConstructor.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestUnoptimizedReaderOnConstructor.java (original)
+++ lucene/dev/trunk/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestUnoptimizedReaderOnConstructor.java Sat Aug 27 13:27:01 2011
@@ -25,7 +25,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.TextField;
 
 /**
  * @since 2009-mar-30 13:15:49
@@ -66,7 +66,7 @@ public class TestUnoptimizedReaderOnCons
 
   private void addDocument(IndexWriter iw, String text) throws IOException {
     Document doc = new Document();
-    doc.add(new Field("field", text, Field.Store.NO, Field.Index.ANALYZED));
+    doc.add(new TextField("field", text));
     iw.addDocument(doc);
   }
 }

Modified: lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Sat Aug 27 13:27:01 2011
@@ -35,23 +35,24 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.OrdTermState;
-import org.apache.lucene.index.TermState;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.FieldsEnum;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermFreqVector;
 import org.apache.lucene.index.TermPositionVector;
+import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermVectorMapper;
-import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.codecs.PerDocValues;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -60,8 +61,8 @@ import org.apache.lucene.search.Similari
 import org.apache.lucene.search.SimilarityProvider;
 import org.apache.lucene.store.RAMDirectory; // for javadocs
 import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Constants; // for javadocs
 
 /**
@@ -240,11 +241,8 @@ public class MemoryIndex {
   /**
    * Convenience method; Tokenizes the given field text and adds the resulting
    * terms to the index; Equivalent to adding an indexed non-keyword Lucene
-   * {@link org.apache.lucene.document.Field} that is
-   * {@link org.apache.lucene.document.Field.Index#ANALYZED tokenized},
-   * {@link org.apache.lucene.document.Field.Store#NO not stored},
-   * {@link org.apache.lucene.document.Field.TermVector#WITH_POSITIONS termVectorStored with positions} (or
-   * {@link org.apache.lucene.document.Field.TermVector#WITH_POSITIONS termVectorStored with positions and offsets}),
+   * {@link org.apache.lucene.document.Field} that is tokenized, not stored,
+   * termVectorStored with positions (or termVectorStored with positions and offsets),
    * 
    * @param fieldName
    *            a name to be associated with the text
@@ -1237,18 +1235,11 @@ public class MemoryIndex {
     }
   
     @Override
-    public Document document(int n) {
+    public void document(int docID, StoredFieldVisitor visitor) {
       if (DEBUG) System.err.println("MemoryIndexReader.document");
-      return new Document(); // there are no stored fields
+      // no-op: there are no stored fields
     }
-
-    //When we convert to JDK 1.5 make this Set<String>
-    @Override
-    public Document document(int n, FieldSelector fieldSelector) throws IOException {
-      if (DEBUG) System.err.println("MemoryIndexReader.document");
-      return new Document(); // there are no stored fields
-    }
-
+    
     @Override
     public boolean hasDeletions() {
       if (DEBUG) System.err.println("MemoryIndexReader.hasDeletions");

Modified: lucene/dev/trunk/lucene/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java (original)
+++ lucene/dev/trunk/lucene/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java Sat Aug 27 13:27:01 2011
@@ -31,6 +31,7 @@ import org.apache.lucene.analysis.MockTo
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.queryparser.classic.QueryParser;
@@ -108,8 +109,8 @@ public class MemoryIndexTest extends Bas
     IndexWriter writer = new IndexWriter(ramdir,
                                          new IndexWriterConfig(TEST_VERSION_CURRENT, analyzer).setCodecProvider(_TestUtil.alwaysCodec("Standard")));
     Document doc = new Document();
-    Field field1 = newField("foo", fooField.toString(), Field.Store.NO, Field.Index.ANALYZED);
-    Field field2 = newField("term", termField.toString(), Field.Store.NO, Field.Index.ANALYZED);
+    Field field1 = newField("foo", fooField.toString(), TextField.TYPE_UNSTORED);
+    Field field2 = newField("term", termField.toString(), TextField.TYPE_UNSTORED);
     doc.add(field1);
     doc.add(field2);
     writer.addDocument(doc);

Copied: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelector.java (from r1161532, lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/FieldSelector.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelector.java?p2=lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelector.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/FieldSelector.java&r1=1161532&r2=1162347&rev=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/FieldSelector.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelector.java Sat Aug 27 13:27:01 2011
@@ -1,5 +1,6 @@
 package org.apache.lucene.document;
 
+
 /**
  * Copyright 2004 The Apache Software Foundation
  *
@@ -18,7 +19,7 @@ package org.apache.lucene.document;
 
 /**
  * Similar to a {@link java.io.FileFilter}, the FieldSelector allows one to make decisions about
- * what Fields get loaded on a {@link Document} by {@link org.apache.lucene.index.IndexReader#document(int,org.apache.lucene.document.FieldSelector)}
+ * what Fields get loaded on a {@link Document} by {@link FieldSelectorVisitor}
  *
  **/
 public interface FieldSelector {

Copied: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorResult.java (from r1161532, lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/FieldSelectorResult.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorResult.java?p2=lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorResult.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/FieldSelectorResult.java&r1=1161532&r2=1162347&rev=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/FieldSelectorResult.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorResult.java Sat Aug 27 13:27:01 2011
@@ -20,39 +20,41 @@ package org.apache.lucene.document;
  *  Provides information about what should be done with this Field 
  *
  **/
+import org.apache.lucene.index.IndexableField; // for javadocs
+
 public enum FieldSelectorResult {
 
     /**
      * Load this {@link Field} every time the {@link Document} is loaded, reading in the data as it is encountered.
-     *  {@link Document#getField(String)} and {@link Document#getFieldable(String)} should not return null.
+     *  {@link Document#getField(String)} should not return null.
      *<p/>
-     * {@link Document#add(Fieldable)} should be called by the Reader.
+     * {@link Document#add(IndexableField)} should be called by the Reader.
      */
   LOAD,
 
     /**
      * Lazily load this {@link Field}.  This means the {@link Field} is valid, but it may not actually contain its data until
-     * invoked.  {@link Document#getField(String)} SHOULD NOT BE USED.  {@link Document#getFieldable(String)} is safe to use and should
-     * return a valid instance of a {@link Fieldable}.
+     * invoked.  {@link Document#getField(String)} is safe to use and should
+     * return a valid instance of a {@link IndexableField}.
      *<p/>
-     * {@link Document#add(Fieldable)} should be called by the Reader.
+     * {@link Document#add(IndexableField)} should be called by the Reader.
      */
   LAZY_LOAD,
 
     /**
-     * Do not load the {@link Field}.  {@link Document#getField(String)} and {@link Document#getFieldable(String)} should return null.
-     * {@link Document#add(Fieldable)} is not called.
+     * Do not load the {@link Field}.  {@link Document#getField(String)} should return null.
+     * {@link Document#add(IndexableField)} is not called.
      * <p/>
-     * {@link Document#add(Fieldable)} should not be called by the Reader.
+     * {@link Document#add(IndexableField)} should not be called by the Reader.
      */
   NO_LOAD,
 
     /**
      * Load this field as in the {@link #LOAD} case, but immediately return from {@link Field} loading for the {@link Document}.  Thus, the
-     * Document may not have its complete set of Fields.  {@link Document#getField(String)} and {@link Document#getFieldable(String)} should
+     * Document may not have its complete set of Fields.  {@link Document#getField(String)} should
      * both be valid for this {@link Field}
      * <p/>
-     * {@link Document#add(Fieldable)} should be called by the Reader.
+     * {@link Document#add(IndexableField)} should be called by the Reader.
      */
   LOAD_AND_BREAK,
 
@@ -67,10 +69,10 @@ public enum FieldSelectorResult {
 
   /**
      * Lazily load this {@link Field}, but do not cache the result.  This means the {@link Field} is valid, but it may not actually contain its data until
-     * invoked.  {@link Document#getField(String)} SHOULD NOT BE USED.  {@link Document#getFieldable(String)} is safe to use and should
-     * return a valid instance of a {@link Fieldable}.
+     * invoked.  {@link Document#getField(String)} is safe to use and should
+     * return a valid instance of a {@link IndexableField}.
      *<p/>
-     * {@link Document#add(Fieldable)} should be called by the Reader.
+     * {@link Document#add(IndexableField)} should be called by the Reader.
      */
   LATENT
 }

Added: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorVisitor.java?rev=1162347&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorVisitor.java (added)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorVisitor.java Sat Aug 27 13:27:01 2011
@@ -0,0 +1,319 @@
+package org.apache.lucene.document;
+
+/**
+ * Copyright 2004 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.Reader;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.document.NumericField.DataType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldReaderException;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+
+/** Create this, passing a legacy {@link FieldSelector} to it, then
+ *  pass this class to {@link IndexReader#document(int,
+ *  StoredFieldVisitor)}, then call {@link #getDocument} to
+ *  retrieve the loaded document.
+
+ *  <p><b>NOTE</b>:  If you use Lazy fields, you should not
+ *  access the returned document after the reader has been
+ *  closed!
+ */
+
+public class FieldSelectorVisitor extends StoredFieldVisitor {
+
+  private final FieldSelector selector;
+  private final Document doc;
+
+  public FieldSelectorVisitor(FieldSelector selector) {
+    this.selector = selector;
+    doc = new Document();
+  }
+
+  public Document getDocument() {
+    return doc;
+  }
+
+  @Override
+  public boolean binaryField(FieldInfo fieldInfo, IndexInput in, int numBytes) throws IOException {
+    final FieldSelectorResult accept = selector.accept(fieldInfo.name);
+    switch (accept) {
+    case LOAD:
+    case LOAD_AND_BREAK:
+      final byte[] b = new byte[numBytes];
+      in.readBytes(b, 0, b.length);
+      doc.add(new BinaryField(fieldInfo.name, b));
+      return accept != FieldSelectorResult.LOAD;
+    case LAZY_LOAD:
+    case LATENT:
+      addFieldLazy(in, fieldInfo, true, accept == FieldSelectorResult.LAZY_LOAD, numBytes);
+      return false;
+    case SIZE:
+    case SIZE_AND_BREAK:
+      in.seek(in.getFilePointer() + numBytes);
+      addFieldSize(fieldInfo, numBytes);
+      return accept != FieldSelectorResult.SIZE;
+    default:
+      // skip
+      in.seek(in.getFilePointer() + numBytes);
+      return false;
+    }
+  }
+
+  @Override
+  public boolean stringField(FieldInfo fieldInfo, IndexInput in, int numUTF8Bytes) throws IOException {
+    final FieldSelectorResult accept = selector.accept(fieldInfo.name);
+    switch (accept) {
+    case LOAD:
+    case LOAD_AND_BREAK:
+      final byte[] b = new byte[numUTF8Bytes];
+      in.readBytes(b, 0, b.length);
+      FieldType ft = new FieldType(TextField.TYPE_STORED);
+      ft.setStoreTermVectors(fieldInfo.storeTermVector);
+      ft.setStoreTermVectorOffsets(fieldInfo.storeOffsetWithTermVector);
+      ft.setStoreTermVectorPositions(fieldInfo.storePositionWithTermVector);
+      doc.add(new Field(fieldInfo.name, ft, new String(b, "UTF-8"))); 
+      return accept != FieldSelectorResult.LOAD;
+    case LAZY_LOAD:
+    case LATENT:
+      addFieldLazy(in, fieldInfo, false, accept == FieldSelectorResult.LAZY_LOAD, numUTF8Bytes);
+      return false;
+    case SIZE:
+    case SIZE_AND_BREAK:
+      in.seek(in.getFilePointer() + numUTF8Bytes);
+      addFieldSize(fieldInfo, 2*numUTF8Bytes);
+      return accept != FieldSelectorResult.SIZE;
+    default:
+      // skip
+      in.seek(in.getFilePointer() + numUTF8Bytes);
+      return false;
+    }
+  }
+
+  @Override
+  public boolean intField(FieldInfo fieldInfo, int value) throws IOException {
+		FieldType ft = new FieldType(NumericField.TYPE_STORED);
+		ft.setIndexed(fieldInfo.isIndexed);
+		ft.setOmitNorms(fieldInfo.omitNorms);
+		ft.setIndexOptions(fieldInfo.indexOptions);
+    return addNumericField(fieldInfo, new NumericField(fieldInfo.name, ft).setIntValue(value));
+  }
+
+  @Override
+  public boolean longField(FieldInfo fieldInfo, long value) throws IOException { 
+		FieldType ft = new FieldType(NumericField.TYPE_STORED);
+		ft.setIndexed(fieldInfo.isIndexed);
+		ft.setOmitNorms(fieldInfo.omitNorms);
+		ft.setIndexOptions(fieldInfo.indexOptions);
+    return addNumericField(fieldInfo, new NumericField(fieldInfo.name, ft).setLongValue(value));
+  }
+
+  @Override
+  public boolean floatField(FieldInfo fieldInfo, float value) throws IOException {
+		FieldType ft = new FieldType(NumericField.TYPE_STORED);
+		ft.setIndexed(fieldInfo.isIndexed);
+		ft.setOmitNorms(fieldInfo.omitNorms);
+		ft.setIndexOptions(fieldInfo.indexOptions);
+    return addNumericField(fieldInfo, new NumericField(fieldInfo.name, ft).setFloatValue(value));
+  }
+
+  @Override
+  public boolean doubleField(FieldInfo fieldInfo, double value) throws IOException {
+		FieldType ft = new FieldType(NumericField.TYPE_STORED);
+		ft.setIndexed(fieldInfo.isIndexed);
+		ft.setOmitNorms(fieldInfo.omitNorms);
+		ft.setIndexOptions(fieldInfo.indexOptions);
+    return addNumericField(fieldInfo, new NumericField(fieldInfo.name, ft).setDoubleValue(value));
+  }
+
+  private boolean addNumericField(FieldInfo fieldInfo, NumericField f) {
+    doc.add(f);
+    final FieldSelectorResult accept = selector.accept(fieldInfo.name);
+    switch (accept) {
+    case LOAD:
+      return false;
+    case LOAD_AND_BREAK:
+      return true;
+    case LAZY_LOAD:
+    case LATENT:
+      return false;
+    case SIZE:
+      return false;
+    case SIZE_AND_BREAK:
+      return true;
+    default:
+      return false;
+    }
+  }
+
+  private void addFieldLazy(IndexInput in, FieldInfo fi, boolean binary, boolean cacheResult, int numBytes) throws IOException {
+    final IndexableField f;
+    final long pointer = in.getFilePointer();
+    // Need to move the pointer ahead by toRead positions
+    in.seek(pointer+numBytes);
+    FieldType ft = new FieldType();
+    ft.setStored(true);
+    ft.setOmitNorms(fi.omitNorms);
+    ft.setIndexOptions(fi.indexOptions);
+    ft.setLazy(true);
+    
+    if (binary) {
+      f = new LazyField(in, fi.name, ft, numBytes, pointer, binary, cacheResult);
+    } else {
+      ft.setStoreTermVectors(fi.storeTermVector);
+      ft.setStoreTermVectorOffsets(fi.storeOffsetWithTermVector);
+      ft.setStoreTermVectorPositions(fi.storePositionWithTermVector);
+      f = new LazyField(in, fi.name, ft, numBytes, pointer, binary, cacheResult);
+    }
+    
+    doc.add(f);
+  }
+
+  // Add the size of field as a byte[] containing the 4 bytes of the integer byte size (high order byte first; char = 2 bytes)
+  // Read just the size -- caller must skip the field content to continue reading fields
+  // Return the size in bytes or chars, depending on field type
+  private void addFieldSize(FieldInfo fi, int numBytes) throws IOException {
+    byte[] sizebytes = new byte[4];
+    sizebytes[0] = (byte) (numBytes>>>24);
+    sizebytes[1] = (byte) (numBytes>>>16);
+    sizebytes[2] = (byte) (numBytes>>> 8);
+    sizebytes[3] = (byte)  numBytes      ;
+    doc.add(new BinaryField(fi.name, sizebytes));
+  }
+
+  /**
+   * A Lazy field implementation that defers loading of fields until asked for, instead of when the Document is
+   * loaded.
+   */
+  private static class LazyField extends Field {
+    private int toRead;
+    private long pointer;
+    private final boolean cacheResult;
+    private final IndexInput in;
+    private boolean isBinary;
+
+    public LazyField(IndexInput in, String name, FieldType ft, int toRead, long pointer, boolean isBinary, boolean cacheResult) {
+      super(name, ft);
+      this.in = in;
+      this.toRead = toRead;
+      this.pointer = pointer;
+      this.isBinary = isBinary;
+      this.cacheResult = cacheResult;
+    }
+
+    @Override
+    public Number numericValue() {
+      return null;
+    }
+
+    @Override
+    public DataType numericDataType() {
+      return null;
+    }
+
+    private IndexInput localFieldsStream;
+
+    private IndexInput getFieldStream() {
+      if (localFieldsStream == null) {
+        localFieldsStream = (IndexInput) in.clone();
+      }
+      return localFieldsStream;
+    }
+
+    /** The value of the field as a Reader, or null.  If null, the String value,
+     * binary value, or TokenStream value is used.  Exactly one of stringValue(), 
+     * readerValue(), getBinaryValue(), and tokenStreamValue() must be set. */
+    @Override
+    public Reader readerValue() {
+      return null;
+    }
+
+    /** The value of the field as a TokenStream, or null.  If null, the Reader value,
+     * String value, or binary value is used. Exactly one of stringValue(), 
+     * readerValue(), getBinaryValue(), and tokenStreamValue() must be set. */
+    @Override
+    public TokenStream tokenStreamValue() {
+      return null;
+    }
+
+    /** The value of the field as a String, or null.  If null, the Reader value,
+     * binary value, or TokenStream value is used.  Exactly one of stringValue(), 
+     * readerValue(), getBinaryValue(), and tokenStreamValue() must be set. */
+    @Override
+    synchronized public String stringValue() {
+      if (isBinary) {
+        return null;
+      } else {
+        if (fieldsData == null) {
+          String result = null;
+          IndexInput localFieldsStream = getFieldStream();
+          try {
+            localFieldsStream.seek(pointer);
+            byte[] bytes = new byte[toRead];
+            localFieldsStream.readBytes(bytes, 0, toRead);
+            result = new String(bytes, "UTF-8");
+          } catch (IOException e) {
+            throw new FieldReaderException(e);
+          }
+          if (cacheResult == true){
+            fieldsData = result;
+          }
+          return result;
+        } else {
+          return (String) fieldsData;
+        }
+      }
+    }
+
+    @Override
+    synchronized public BytesRef binaryValue() {
+      if (isBinary) {
+        if (fieldsData == null) {
+          // Allocate new buffer if result is null or too small
+          final byte[] b = new byte[toRead];
+   
+          IndexInput localFieldsStream = getFieldStream();
+
+          // Throw this IOException since IndexReader.document does so anyway, so probably not that big of a change for people
+          // since they are already handling this exception when getting the document
+          try {
+            localFieldsStream.seek(pointer);
+            localFieldsStream.readBytes(b, 0, toRead);
+          } catch (IOException e) {
+            throw new FieldReaderException(e);
+          }
+
+          final BytesRef result = new BytesRef(b);
+          result.length = toRead;
+          if (cacheResult == true){
+            fieldsData = result;
+          }
+          return result;
+        } else {
+          return (BytesRef) fieldsData;
+        }
+      } else {
+        return null;
+      }
+    }
+  }
+}

Copied: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java (from r1161532, lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java?p2=lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java&r1=1161532&r2=1162347&rev=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java Sat Aug 27 13:27:01 2011
@@ -26,4 +26,4 @@ public class LoadFirstFieldSelector impl
   public FieldSelectorResult accept(String fieldName) {
     return FieldSelectorResult.LOAD_AND_BREAK;
   }
-}
\ No newline at end of file
+}

Copied: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/SetBasedFieldSelector.java (from r1161532, lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/SetBasedFieldSelector.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/SetBasedFieldSelector.java?p2=lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/SetBasedFieldSelector.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/SetBasedFieldSelector.java&r1=1161532&r2=1162347&rev=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/document/SetBasedFieldSelector.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/document/SetBasedFieldSelector.java Sat Aug 27 13:27:01 2011
@@ -1,6 +1,5 @@
 package org.apache.lucene.document;
 
-import java.util.Set;
 /**
  * Copyright 2004 The Apache Software Foundation
  *
@@ -17,10 +16,14 @@ import java.util.Set;
  * limitations under the License.
  */
 
+import java.util.Set;
+
+
 /**
  * Declare what fields to load normally and what fields to load lazily
  *
  **/
+
 public class SetBasedFieldSelector implements FieldSelector {
   
   private Set<String> fieldsToLoad;
@@ -55,4 +58,4 @@ public class SetBasedFieldSelector imple
     }                                           
     return result;
   }
-}
\ No newline at end of file
+}

Modified: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java Sat Aug 27 13:27:01 2011
@@ -35,7 +35,7 @@ import org.apache.lucene.util.ReaderUtil
  * 
  * If Similarity class is specified, uses its computeNorm method to set norms.
  * If -n command line argument is used, removed field norms, as if 
- * {@link org.apache.lucene.document.Field.Index}.NO_NORMS was used.
+ * {@link org.apache.lucene.document.FieldType#setOmitNorms(boolean)} was used.
  *
  * <p>
  * NOTE: This will overwrite any length normalization or field/document boosts.

Added: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribFieldsReader.java?rev=1162347&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribFieldsReader.java (added)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribFieldsReader.java Sat Aug 27 13:27:01 2011
@@ -0,0 +1,318 @@
+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.io.File;
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldSelector;
+import org.apache.lucene.document.FieldSelectorResult;
+import org.apache.lucene.document.FieldSelectorVisitor;
+import org.apache.lucene.document.LoadFirstFieldSelector;
+import org.apache.lucene.document.SetBasedFieldSelector;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+
+public class TestContribFieldsReader extends LuceneTestCase {
+  private static Directory dir;
+  private static org.apache.lucene.document.Document testDoc = new org.apache.lucene.document.Document();
+  private static FieldInfos fieldInfos = null;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    fieldInfos = new FieldInfos();
+    DocHelper.setupDoc(testDoc);
+    _TestUtil.add(testDoc, fieldInfos);
+    dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy());
+    ((LogMergePolicy) conf.getMergePolicy()).setUseCompoundFile(false);
+    IndexWriter writer = new IndexWriter(dir, conf);
+    writer.addDocument(testDoc);
+    writer.close();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    dir.close();
+    dir = null;
+    fieldInfos = null;
+    testDoc = null;
+  }
+
+  private Document getDocument(IndexReader ir, int docID, FieldSelector selector) throws IOException {
+    final FieldSelectorVisitor visitor = new FieldSelectorVisitor(selector);
+    ir.document(docID, visitor);
+    return visitor.getDocument();
+  }
+
+  public void testLazyFields() throws Exception {
+    assertTrue(dir != null);
+    assertTrue(fieldInfos != null);
+    IndexReader reader = IndexReader.open(dir);
+    Set<String> loadFieldNames = new HashSet<String>();
+    loadFieldNames.add(DocHelper.TEXT_FIELD_1_KEY);
+    loadFieldNames.add(DocHelper.TEXT_FIELD_UTF1_KEY);
+    Set<String> lazyFieldNames = new HashSet<String>();
+    //new String[]{DocHelper.LARGE_LAZY_FIELD_KEY, DocHelper.LAZY_FIELD_KEY, DocHelper.LAZY_FIELD_BINARY_KEY};
+    lazyFieldNames.add(DocHelper.LARGE_LAZY_FIELD_KEY);
+    lazyFieldNames.add(DocHelper.LAZY_FIELD_KEY);
+    lazyFieldNames.add(DocHelper.LAZY_FIELD_BINARY_KEY);
+    lazyFieldNames.add(DocHelper.TEXT_FIELD_UTF2_KEY);
+    SetBasedFieldSelector fieldSelector = new SetBasedFieldSelector(loadFieldNames, lazyFieldNames);
+    Document doc = getDocument(reader, 0, fieldSelector);
+    assertTrue("doc is null and it shouldn't be", doc != null);
+    IndexableField field = doc.getField(DocHelper.LAZY_FIELD_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertTrue("field is not lazy and it should be", field.getClass().getSimpleName().equals("LazyField"));
+    String value = field.stringValue();
+    assertTrue("value is null and it shouldn't be", value != null);
+    assertTrue(value + " is not equal to " + DocHelper.LAZY_FIELD_TEXT, value.equals(DocHelper.LAZY_FIELD_TEXT) == true);
+    assertTrue("calling stringValue() twice should give same reference", field.stringValue() == field.stringValue());
+
+    field = doc.getField(DocHelper.TEXT_FIELD_1_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertFalse("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
+    field = doc.getField(DocHelper.TEXT_FIELD_UTF1_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertFalse("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
+    assertTrue(field.stringValue() + " is not equal to " + DocHelper.FIELD_UTF1_TEXT, field.stringValue().equals(DocHelper.FIELD_UTF1_TEXT) == true);
+
+    field = doc.getField(DocHelper.TEXT_FIELD_UTF2_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertTrue("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
+    assertTrue(field.stringValue() + " is not equal to " + DocHelper.FIELD_UTF2_TEXT, field.stringValue().equals(DocHelper.FIELD_UTF2_TEXT) == true);
+
+    field = doc.getField(DocHelper.LAZY_FIELD_BINARY_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertTrue("stringValue isn't null for lazy binary field", field.stringValue() == null);
+
+    byte [] bytes = field.binaryValue().bytes;
+    assertTrue("bytes is null and it shouldn't be", bytes != null);
+    assertTrue("", DocHelper.LAZY_FIELD_BINARY_BYTES.length == bytes.length);
+    assertTrue("calling binaryValue() twice should give same reference", field.binaryValue().bytes == field.binaryValue().bytes);
+    for (int i = 0; i < bytes.length; i++) {
+      assertTrue("byte[" + i + "] is mismatched", bytes[i] == DocHelper.LAZY_FIELD_BINARY_BYTES[i]);
+
+    }
+    reader.close();
+  }
+
+  public void testLatentFields() throws Exception {
+    assertTrue(dir != null);
+    assertTrue(fieldInfos != null);
+    IndexReader reader = IndexReader.open(dir);
+    Set<String> loadFieldNames = new HashSet<String>();
+    loadFieldNames.add(DocHelper.TEXT_FIELD_1_KEY);
+    loadFieldNames.add(DocHelper.TEXT_FIELD_UTF1_KEY);
+    Set<String> lazyFieldNames = new HashSet<String>();
+    //new String[]{DocHelper.LARGE_LAZY_FIELD_KEY, DocHelper.LAZY_FIELD_KEY, DocHelper.LAZY_FIELD_BINARY_KEY};
+    lazyFieldNames.add(DocHelper.LARGE_LAZY_FIELD_KEY);
+    lazyFieldNames.add(DocHelper.LAZY_FIELD_KEY);
+    lazyFieldNames.add(DocHelper.LAZY_FIELD_BINARY_KEY);
+    lazyFieldNames.add(DocHelper.TEXT_FIELD_UTF2_KEY);
+
+    // Use LATENT instead of LAZY
+    SetBasedFieldSelector fieldSelector = new SetBasedFieldSelector(loadFieldNames, lazyFieldNames) {
+        @Override
+        public FieldSelectorResult accept(String fieldName) {
+          final FieldSelectorResult result = super.accept(fieldName);
+          if (result == FieldSelectorResult.LAZY_LOAD) {
+            return FieldSelectorResult.LATENT;
+          } else {
+            return result;
+          }
+        }
+      };
+
+    Document doc = getDocument(reader, 0, fieldSelector);
+    assertTrue("doc is null and it shouldn't be", doc != null);
+    IndexableField field = doc.getField(DocHelper.LAZY_FIELD_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertTrue("field is not lazy and it should be", field.getClass().getSimpleName().equals("LazyField"));
+    String value = field.stringValue();
+    assertTrue("value is null and it shouldn't be", value != null);
+    assertTrue(value + " is not equal to " + DocHelper.LAZY_FIELD_TEXT, value.equals(DocHelper.LAZY_FIELD_TEXT) == true);
+    assertTrue("calling stringValue() twice should give different references", field.stringValue() != field.stringValue());
+
+    field = doc.getField(DocHelper.TEXT_FIELD_1_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertFalse("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
+    assertTrue("calling stringValue() twice should give same reference", field.stringValue() == field.stringValue());
+
+    field = doc.getField(DocHelper.TEXT_FIELD_UTF1_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertFalse("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
+    assertTrue(field.stringValue() + " is not equal to " + DocHelper.FIELD_UTF1_TEXT, field.stringValue().equals(DocHelper.FIELD_UTF1_TEXT) == true);
+    assertTrue("calling stringValue() twice should give same reference", field.stringValue() == field.stringValue());
+
+    field = doc.getField(DocHelper.TEXT_FIELD_UTF2_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertTrue("Field is lazy and it should not be", field.getClass().getSimpleName().equals("LazyField"));
+    assertTrue(field.stringValue() + " is not equal to " + DocHelper.FIELD_UTF2_TEXT, field.stringValue().equals(DocHelper.FIELD_UTF2_TEXT) == true);
+    assertTrue("calling stringValue() twice should give different references", field.stringValue() != field.stringValue());
+
+    field = doc.getField(DocHelper.LAZY_FIELD_BINARY_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertTrue("stringValue isn't null for lazy binary field", field.stringValue() == null);
+    assertTrue("calling binaryValue() twice should give different references", field.binaryValue().bytes != field.binaryValue().bytes);
+
+    byte [] bytes = field.binaryValue().bytes;
+    assertTrue("bytes is null and it shouldn't be", bytes != null);
+    assertTrue("", DocHelper.LAZY_FIELD_BINARY_BYTES.length == bytes.length);
+    for (int i = 0; i < bytes.length; i++) {
+      assertTrue("byte[" + i + "] is mismatched", bytes[i] == DocHelper.LAZY_FIELD_BINARY_BYTES[i]);
+
+    }
+    reader.close();
+  }
+
+  public void testLoadFirst() throws Exception {
+    assertTrue(dir != null);
+    assertTrue(fieldInfos != null);
+    IndexReader reader = IndexReader.open(dir);
+    LoadFirstFieldSelector fieldSelector = new LoadFirstFieldSelector();
+    Document doc = getDocument(reader, 0, fieldSelector);
+    assertTrue("doc is null and it shouldn't be", doc != null);
+    int count = 0;
+    List<IndexableField> l = doc.getFields();
+    for (final IndexableField IndexableField : l ) {
+      Field field = (Field) IndexableField;
+
+      assertTrue("field is null and it shouldn't be", field != null);
+      String sv = field.stringValue();
+      assertTrue("sv is null and it shouldn't be", sv != null);
+      count++;
+    }
+    assertTrue(count + " does not equal: " + 1, count == 1);
+    reader.close();
+  }
+
+  /**
+   * Not really a test per se, but we should have some way of assessing whether this is worthwhile.
+   * <p/>
+   * Must test using a File based directory
+   *
+   * @throws Exception
+   */
+  public void testLazyPerformance() throws Exception {
+    String userName = System.getProperty("user.name");
+    File file = _TestUtil.getTempDir("lazyDir" + userName);
+    Directory tmpDir = newFSDirectory(file);
+    assertTrue(tmpDir != null);
+
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE).setMergePolicy(newLogMergePolicy());
+    ((LogMergePolicy) conf.getMergePolicy()).setUseCompoundFile(false);
+    IndexWriter writer = new IndexWriter(tmpDir, conf);
+    writer.addDocument(testDoc);
+    writer.close();
+
+    assertTrue(fieldInfos != null);
+    long lazyTime = 0;
+    long regularTime = 0;
+    int length = 10;
+    Set<String> lazyFieldNames = new HashSet<String>();
+    lazyFieldNames.add(DocHelper.LARGE_LAZY_FIELD_KEY);
+    SetBasedFieldSelector fieldSelector = new SetBasedFieldSelector(Collections. <String> emptySet(), lazyFieldNames);
+
+    for (int i = 0; i < length; i++) {
+      IndexReader reader = IndexReader.open(tmpDir);
+
+      Document doc;
+      doc = reader.document(0);//Load all of them
+      assertTrue("doc is null and it shouldn't be", doc != null);
+      IndexableField field = doc.getField(DocHelper.LARGE_LAZY_FIELD_KEY);
+      assertTrue("field is null and it shouldn't be", field != null);
+      assertFalse("field is lazy", field.getClass().getSimpleName().equals("LazyField"));
+      String value;
+      long start;
+      long finish;
+      start = System.currentTimeMillis();
+      //On my machine this was always 0ms.
+      value = field.stringValue();
+      finish = System.currentTimeMillis();
+      assertTrue("value is null and it shouldn't be", value != null);
+      regularTime += (finish - start);
+      reader.close();
+      reader = null;
+      doc = null;
+      //Hmmm, are we still in cache???
+      System.gc();
+      reader = IndexReader.open(tmpDir);
+      doc = getDocument(reader, 0, fieldSelector);
+      field = doc.getField(DocHelper.LARGE_LAZY_FIELD_KEY);
+      assertTrue("field is not lazy", field.getClass().getSimpleName().equals("LazyField"));
+      start = System.currentTimeMillis();
+      //On my machine this took around 50 - 70ms
+      value = field.stringValue();
+      finish = System.currentTimeMillis();
+      assertTrue("value is null and it shouldn't be", value != null);
+      lazyTime += (finish - start);
+      reader.close();
+
+    }
+    tmpDir.close();
+    if (VERBOSE) {
+      System.out.println("Average Non-lazy time (should be very close to zero): " + regularTime / length + " ms for " + length + " reads");
+      System.out.println("Average Lazy Time (should be greater than zero): " + lazyTime / length + " ms for " + length + " reads");
+    }
+  }
+  
+  public void testLoadSize() throws IOException {
+    IndexReader reader = IndexReader.open(dir);
+    Document doc;
+    
+    doc = getDocument(reader, 0, new FieldSelector(){
+      public FieldSelectorResult accept(String fieldName) {
+        if (fieldName.equals(DocHelper.TEXT_FIELD_1_KEY) ||
+            fieldName.equals(DocHelper.LAZY_FIELD_BINARY_KEY))
+          return FieldSelectorResult.SIZE;
+        else if (fieldName.equals(DocHelper.TEXT_FIELD_3_KEY))
+          return FieldSelectorResult.LOAD;
+        else
+          return FieldSelectorResult.NO_LOAD;
+      }
+    });
+    IndexableField f1 = doc.getField(DocHelper.TEXT_FIELD_1_KEY);
+    IndexableField f3 = doc.getField(DocHelper.TEXT_FIELD_3_KEY);
+    IndexableField fb = doc.getField(DocHelper.LAZY_FIELD_BINARY_KEY);
+    assertTrue(f1.binaryValue()!=null);
+    assertTrue(f3.binaryValue()==null);
+    assertTrue(fb.binaryValue()!=null);
+    assertSizeEquals(2*DocHelper.FIELD_1_TEXT.length(), f1.binaryValue().bytes);
+    assertEquals(DocHelper.FIELD_3_TEXT, f3.stringValue());
+    assertSizeEquals(DocHelper.LAZY_FIELD_BINARY_BYTES.length, fb.binaryValue().bytes);
+    
+    reader.close();
+  }
+  
+  private void assertSizeEquals(int size, byte[] sizebytes) {
+    assertEquals((byte) (size>>>24), sizebytes[0]);
+    assertEquals((byte) (size>>>16), sizebytes[1]);
+    assertEquals((byte) (size>>> 8), sizebytes[2]);
+    assertEquals((byte)  size      , sizebytes[3]);
+  }
+}

Added: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribIndexReader.java?rev=1162347&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribIndexReader.java (added)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribIndexReader.java Sat Aug 27 13:27:01 2011
@@ -0,0 +1,169 @@
+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.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.BinaryField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldSelector;
+import org.apache.lucene.document.FieldSelectorVisitor;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.SetBasedFieldSelector;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestContribIndexReader extends LuceneTestCase {
+  private Document getDocument(IndexReader ir, int docID, FieldSelector selector)  throws IOException {
+    final FieldSelectorVisitor visitor = new FieldSelectorVisitor(selector);
+    ir.document(docID, visitor);
+    return visitor.getDocument();
+  }
+
+  static void addDoc(IndexWriter writer, String value) throws IOException {
+    Document doc = new Document();
+    doc.add(newField("content", value, TextField.TYPE_UNSTORED));
+    writer.addDocument(doc);
+  }
+
+  static void addDocumentWithFields(IndexWriter writer) throws IOException {
+    Document doc = new Document();
+        
+    FieldType customType3 = new FieldType();
+    customType3.setStored(true);
+    doc.add(newField("keyword", "test1", StringField.TYPE_STORED));
+    doc.add(newField("text", "test1", TextField.TYPE_STORED));
+    doc.add(newField("unindexed", "test1", customType3));
+    doc.add(new TextField("unstored","test1"));
+    writer.addDocument(doc);
+  }
+
+
+  static void addDocumentWithDifferentFields(IndexWriter writer) throws IOException {
+    Document doc = new Document();
+      
+    FieldType customType3 = new FieldType();
+    customType3.setStored(true);
+    doc.add(newField("keyword2", "test1", StringField.TYPE_STORED));
+    doc.add(newField("text2", "test1", TextField.TYPE_STORED));
+    doc.add(newField("unindexed2", "test1", customType3));
+    doc.add(new TextField("unstored2","test1"));
+    writer.addDocument(doc);
+  }
+
+  static void addDocumentWithTermVectorFields(IndexWriter writer) throws IOException {
+    Document doc = new Document();
+    FieldType customType5 = new FieldType(TextField.TYPE_STORED);
+    customType5.setStoreTermVectors(true);
+    FieldType customType6 = new FieldType(TextField.TYPE_STORED);
+    customType6.setStoreTermVectors(true);
+    customType6.setStoreTermVectorOffsets(true);
+    FieldType customType7 = new FieldType(TextField.TYPE_STORED);
+    customType7.setStoreTermVectors(true);
+    customType7.setStoreTermVectorPositions(true);
+    FieldType customType8 = new FieldType(TextField.TYPE_STORED);
+    customType8.setStoreTermVectors(true);
+    customType8.setStoreTermVectorOffsets(true);
+    customType8.setStoreTermVectorPositions(true);
+    doc.add(newField("tvnot","tvnot",TextField.TYPE_STORED));
+    doc.add(newField("termvector","termvector",customType5));
+    doc.add(newField("tvoffset","tvoffset", customType6));
+    doc.add(newField("tvposition","tvposition", customType7));
+    doc.add(newField("tvpositionoffset","tvpositionoffset", customType8));
+        
+    writer.addDocument(doc);
+  }
+
+  public void testBinaryFields() throws IOException {
+    Directory dir = newDirectory();
+    byte[] bin = new byte[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9};
+        
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
+        
+    for (int i = 0; i < 10; i++) {
+      addDoc(writer, "document number " + (i + 1));
+      addDocumentWithFields(writer);
+      addDocumentWithDifferentFields(writer);
+      addDocumentWithTermVectorFields(writer);
+    }
+    writer.close();
+    writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND).setMergePolicy(newLogMergePolicy()));
+    Document doc = new Document();
+    doc.add(new BinaryField("bin1", bin));
+    doc.add(new TextField("junk", "junk text"));
+    writer.addDocument(doc);
+    writer.close();
+    IndexReader reader = IndexReader.open(dir, false);
+    Document doc2 = reader.document(reader.maxDoc() - 1);
+    IndexableField[] fields = doc2.getFields("bin1");
+    assertNotNull(fields);
+    assertEquals(1, fields.length);
+    Field b1 = (Field) fields[0];
+    assertTrue(b1.isBinary());
+    BytesRef bytesRef = b1.binaryValue();
+    assertEquals(bin.length, bytesRef.length);
+    for (int i = 0; i < bin.length; i++) {
+      assertEquals(bin[i], bytesRef.bytes[i + bytesRef.offset]);
+    }
+    Set<String> lazyFields = new HashSet<String>();
+    lazyFields.add("bin1");
+    FieldSelector sel = new SetBasedFieldSelector(new HashSet<String>(), lazyFields);
+    doc2 = getDocument(reader, reader.maxDoc() - 1, sel);
+    fields = doc2.getFields("bin1");
+    assertNotNull(fields);
+    assertEquals(1, fields.length);
+    IndexableField fb1 = fields[0];
+    assertTrue(fb1.binaryValue()!=null);
+    bytesRef = fb1.binaryValue();
+    assertEquals(bin.length, bytesRef.bytes.length);
+    assertEquals(bin.length, bytesRef.length);
+    for (int i = 0; i < bin.length; i++) {
+      assertEquals(bin[i], bytesRef.bytes[i + bytesRef.offset]);
+    }
+    reader.close();
+    // force optimize
+
+
+    writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND).setMergePolicy(newLogMergePolicy()));
+    writer.optimize();
+    writer.close();
+    reader = IndexReader.open(dir, false);
+    doc2 = reader.document(reader.maxDoc() - 1);
+    fields = doc2.getFields("bin1");
+    assertNotNull(fields);
+    assertEquals(1, fields.length);
+    b1 = (Field) fields[0];
+    assertTrue(b1.isBinary());
+    bytesRef = b1.binaryValue();
+    assertEquals(bin.length, bytesRef.length);
+    for (int i = 0; i < bin.length; i++) {
+      assertEquals(bin[i], bytesRef.bytes[i + bytesRef.offset]);
+    }
+    reader.close();
+    dir.close();
+  }
+}

Added: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribParallelReader.java?rev=1162347&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribParallelReader.java (added)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribParallelReader.java Sat Aug 27 13:27:01 2011
@@ -0,0 +1,149 @@
+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.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FieldSelector;
+import org.apache.lucene.document.FieldSelectorVisitor;
+import org.apache.lucene.document.MapFieldSelector;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestContribParallelReader extends LuceneTestCase {
+
+  private IndexSearcher parallel;
+  private IndexSearcher single;
+  private Directory dir, dir1, dir2;
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    single = single(random);
+    parallel = parallel(random);
+  }
+  
+  @Override
+  public void tearDown() throws Exception {
+    single.getIndexReader().close();
+    single.close();
+    parallel.getIndexReader().close();
+    parallel.close();
+    dir.close();
+    dir1.close();
+    dir2.close();
+    super.tearDown();
+  }
+
+  // Fields 1-4 indexed together:
+  private IndexSearcher single(Random random) throws IOException {
+    dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)));
+    Document d1 = new Document();
+    d1.add(newField("f1", "v1", TextField.TYPE_STORED));
+    d1.add(newField("f2", "v1", TextField.TYPE_STORED));
+    d1.add(newField("f3", "v1", TextField.TYPE_STORED));
+    d1.add(newField("f4", "v1", TextField.TYPE_STORED));
+    w.addDocument(d1);
+    Document d2 = new Document();
+    d2.add(newField("f1", "v2", TextField.TYPE_STORED));
+    d2.add(newField("f2", "v2", TextField.TYPE_STORED));
+    d2.add(newField("f3", "v2", TextField.TYPE_STORED));
+    d2.add(newField("f4", "v2", TextField.TYPE_STORED));
+    w.addDocument(d2);
+    w.close();
+
+    return new IndexSearcher(dir, false);
+  }
+
+  // Fields 1 & 2 in one index, 3 & 4 in other, with ParallelReader:
+  private IndexSearcher parallel(Random random) throws IOException {
+    dir1 = getDir1(random);
+    dir2 = getDir2(random);
+    ParallelReader pr = new ParallelReader();
+    pr.add(IndexReader.open(dir1, false));
+    pr.add(IndexReader.open(dir2, false));
+    return newSearcher(pr);
+  }
+
+  private Document getDocument(IndexReader ir, int docID, FieldSelector selector) throws IOException {
+    final FieldSelectorVisitor visitor = new FieldSelectorVisitor(selector);
+    ir.document(docID, visitor);
+    return visitor.getDocument();
+  }
+
+  public void testDocument() throws IOException {
+    Directory dir1 = getDir1(random);
+    Directory dir2 = getDir2(random);
+    ParallelReader pr = new ParallelReader();
+    pr.add(IndexReader.open(dir1, false));
+    pr.add(IndexReader.open(dir2, false));
+
+    Document doc11 = getDocument(pr, 0, new MapFieldSelector("f1"));
+    Document doc24 = getDocument(pr, 1, new MapFieldSelector(Arrays.asList("f4")));
+    Document doc223 = getDocument(pr, 1, new MapFieldSelector("f2", "f3"));
+    
+    assertEquals(1, doc11.getFields().size());
+    assertEquals(1, doc24.getFields().size());
+    assertEquals(2, doc223.getFields().size());
+    
+    assertEquals("v1", doc11.get("f1"));
+    assertEquals("v2", doc24.get("f4"));
+    assertEquals("v2", doc223.get("f2"));
+    assertEquals("v2", doc223.get("f3"));
+    pr.close();
+    dir1.close();
+    dir2.close();
+  }
+
+  private Directory getDir1(Random random) throws IOException {
+    Directory dir1 = newDirectory();
+    IndexWriter w1 = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)));
+    Document d1 = new Document();
+    d1.add(newField("f1", "v1", TextField.TYPE_STORED));
+    d1.add(newField("f2", "v1", TextField.TYPE_STORED));
+    w1.addDocument(d1);
+    Document d2 = new Document();
+    d2.add(newField("f1", "v2", TextField.TYPE_STORED));
+    d2.add(newField("f2", "v2", TextField.TYPE_STORED));
+    w1.addDocument(d2);
+    w1.close();
+    return dir1;
+  }
+
+  private Directory getDir2(Random random) throws IOException {
+    Directory dir2 = newDirectory();
+    IndexWriter w2 = new IndexWriter(dir2, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)));
+    Document d3 = new Document();
+    d3.add(newField("f3", "v1", TextField.TYPE_STORED));
+    d3.add(newField("f4", "v1", TextField.TYPE_STORED));
+    w2.addDocument(d3);
+    Document d4 = new Document();
+    d4.add(newField("f3", "v2", TextField.TYPE_STORED));
+    d4.add(newField("f4", "v2", TextField.TYPE_STORED));
+    w2.addDocument(d4);
+    w2.close();
+    return dir2;
+  }
+}

Modified: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java Sat Aug 27 13:27:01 2011
@@ -22,7 +22,8 @@ import java.util.Arrays;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.DefaultSimilarity;
@@ -65,13 +66,15 @@ public class TestFieldNormModifier exten
     
     for (int i = 0; i < NUM_DOCS; i++) {
       Document d = new Document();
-      d.add(newField("field", "word", Field.Store.YES, Field.Index.ANALYZED));
-      d.add(newField("nonorm", "word", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS));
-      d.add(newField("untokfield", "20061212 20071212", Field.Store.YES, Field.Index.ANALYZED));
+      
+      d.add(newField("field", "word", TextField.TYPE_STORED));
+
+      d.add(newField("nonorm", "word", StringField.TYPE_STORED));
+      d.add(newField("untokfield", "20061212 20071212", TextField.TYPE_STORED));
       
       for (int j = 1; j <= i; j++) {
-        d.add(newField("field", "crap", Field.Store.YES, Field.Index.ANALYZED));
-        d.add(newField("nonorm", "more words", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS));
+        d.add(newField("field", "crap", TextField.TYPE_STORED));
+        d.add(newField("nonorm", "more words", StringField.TYPE_STORED));
       }
       writer.addDocument(d);
     }

Modified: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestIndexSplitter.java Sat Aug 27 13:27:01 2011
@@ -21,6 +21,7 @@ import java.io.File;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
@@ -112,10 +113,10 @@ public class TestIndexSplitter extends L
     Directory fsDir = newFSDirectory(indexPath);
     IndexWriter indexWriter = new IndexWriter(fsDir, iwConfig);
     Document doc = new Document();
-    doc.add(new Field("content", "doc 1", Field.Store.YES, Field.Index.ANALYZED_NO_NORMS));
+    doc.add(new Field("content", StringField.TYPE_STORED, "doc 1"));
     indexWriter.addDocument(doc);
     doc = new Document();
-    doc.add(new Field("content", "doc 2", Field.Store.YES, Field.Index.ANALYZED_NO_NORMS));
+    doc.add(new Field("content", StringField.TYPE_STORED, "doc 2"));
     indexWriter.addDocument(doc);
     indexWriter.close();
     fsDir.close();

Copied: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestLazyBug.java (from r1161532, lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLazyBug.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestLazyBug.java?p2=lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestLazyBug.java&p1=lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLazyBug.java&r1=1161532&r2=1162347&rev=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestLazyBug.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestLazyBug.java Sat Aug 27 13:27:01 2011
@@ -22,7 +22,11 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.*;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FieldSelector;
+import org.apache.lucene.document.FieldSelectorResult;
+import org.apache.lucene.document.FieldSelectorVisitor;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.AfterClass;
@@ -87,8 +91,7 @@ public class TestLazyBug extends LuceneT
           doc.add(newField("f"+f,
                             data[f % data.length]
                             + '#' + data[random.nextInt(data.length)],
-                            Field.Store.NO,
-                            Field.Index.ANALYZED));
+                            TextField.TYPE_UNSTORED));
         }
         writer.addDocument(doc);
       }
@@ -102,12 +105,14 @@ public class TestLazyBug extends LuceneT
   public void doTest(int[] docs) throws Exception {
     IndexReader reader = IndexReader.open(directory, true);
     for (int i = 0; i < docs.length; i++) {
-      Document d = reader.document(docs[i], SELECTOR);
+      final FieldSelectorVisitor visitor = new FieldSelectorVisitor(SELECTOR);
+      reader.document(docs[i], visitor);
+      Document d = visitor.getDocument();
       d.get(MAGIC_FIELD);
 
-      List<Fieldable> fields = d.getFields();
-      for (Iterator<Fieldable> fi = fields.iterator(); fi.hasNext(); ) {
-        Fieldable f=null;
+      List<IndexableField> fields = d.getFields();
+      for (Iterator<IndexableField> fi = fields.iterator(); fi.hasNext(); ) {
+        IndexableField f=null;
         try {
           f =  fi.next();
           String fname = f.name();
@@ -136,5 +141,4 @@ public class TestLazyBug extends LuceneT
   public void testLazyBroken() throws Exception {
     doTest(new int[] { NUM_DOCS/2, NUM_DOCS-1 });
   }
-
 }

Modified: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestMultiPassIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestMultiPassIndexSplitter.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestMultiPassIndexSplitter.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestMultiPassIndexSplitter.java Sat Aug 27 13:27:01 2011
@@ -18,10 +18,11 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
 
 public class TestMultiPassIndexSplitter extends LuceneTestCase {
   IndexReader input;
@@ -36,8 +37,8 @@ public class TestMultiPassIndexSplitter 
     Document doc;
     for (int i = 0; i < NUM_DOCS; i++) {
       doc = new Document();
-      doc.add(newField("id", i + "", Field.Store.YES, Field.Index.NOT_ANALYZED));
-      doc.add(newField("f", i + " " + i, Field.Store.YES, Field.Index.ANALYZED));
+      doc.add(newField("id", i + "", StringField.TYPE_STORED));
+      doc.add(newField("f", i + " " + i, TextField.TYPE_STORED));
       w.addDocument(doc);
     }
     w.close();

Modified: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestNRTManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestNRTManager.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestNRTManager.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestNRTManager.java Sat Aug 27 13:27:01 2011
@@ -33,7 +33,8 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
@@ -77,17 +78,12 @@ public class TestNRTManager extends Luce
   // TODO: is there a pre-existing way to do this!!!
   private Document cloneDoc(Document doc1) {
     final Document doc2 = new Document();
-    for(Fieldable f : doc1.getFields()) {
+    for(IndexableField f : doc1) {
       Field field1 = (Field) f;
       
       Field field2 = new Field(field1.name(),
-                               field1.stringValue(),
-                               field1.isStored() ? Field.Store.YES : Field.Store.NO,
-                               field1.isIndexed() ? (field1.isTokenized() ? Field.Index.ANALYZED : Field.Index.NOT_ANALYZED) : Field.Index.NO);
-      if (field1.getOmitNorms()) {
-        field2.setOmitNorms(true);
-      }
-      field2.setIndexOptions(field1.getIndexOptions());
+                              ((Field) f).getFieldType(),
+                               field1.stringValue());
       doc2.add(field2);
     }
 
@@ -240,7 +236,7 @@ public class TestNRTManager extends Luce
                 final String addedField;
                 if (random.nextBoolean()) {
                   addedField = "extra" + random.nextInt(10);
-                  doc.add(new Field(addedField, "a random field", Field.Store.NO, Field.Index.ANALYZED));
+                  doc.add(new TextField(addedField, "a random field"));
                 } else {
                   addedField = null;
                 }
@@ -262,7 +258,7 @@ public class TestNRTManager extends Luce
                       packID = packCount.getAndIncrement() + "";
                     }
 
-                    final Field packIDField = newField("packID", packID, Field.Store.YES, Field.Index.NOT_ANALYZED);
+                    final Field packIDField = newField("packID", packID, StringField.TYPE_STORED);
                     final List<String> docIDs = new ArrayList<String>();
                     final SubDocs subDocs = new SubDocs(packID, docIDs);
                     final List<Document> docsList = new ArrayList<Document>();

Modified: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestPKIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestPKIndexSplitter.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestPKIndexSplitter.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestPKIndexSplitter.java Sat Aug 27 13:27:01 2011
@@ -23,8 +23,8 @@ import java.text.NumberFormat;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Index;
-import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
@@ -102,15 +102,15 @@ public class TestPKIndexSplitter extends
     StringBuilder sb = new StringBuilder();
     Document doc = new Document();
     String id = format.format(n);
-    doc.add(newField("id", id, Store.YES, Index.NOT_ANALYZED));
-    doc.add(newField("indexname", indexName, Store.YES, Index.NOT_ANALYZED));
+    doc.add(newField("id", id, StringField.TYPE_STORED));
+    doc.add(newField("indexname", indexName, StringField.TYPE_STORED));
     sb.append("a");
     sb.append(n);
-    doc.add(newField("field1", sb.toString(), Store.YES, Index.ANALYZED));
+    doc.add(newField("field1", sb.toString(), TextField.TYPE_STORED));
     sb.append(" b");
     sb.append(n);
     for (int i = 1; i < numFields; i++) {
-      doc.add(newField("field" + (i + 1), sb.toString(), Store.YES, Index.ANALYZED));
+      doc.add(newField("field" + (i + 1), sb.toString(), TextField.TYPE_STORED));
     }
     return doc;
   }

Modified: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestTermVectorAccessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestTermVectorAccessor.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestTermVectorAccessor.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/TestTermVectorAccessor.java Sat Aug 27 13:27:01 2011
@@ -2,7 +2,8 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
@@ -30,33 +31,42 @@ public class TestTermVectorAccessor exte
     Document doc;
 
     doc = new Document();
-    doc.add(newField("a", "a b a c a d a e a f a g a h a", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
-    doc.add(newField("b", "a b c b d b e b f b g b h b", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
-    doc.add(newField("c", "a c b c d c e c f c g c h c", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
+    FieldType customType = new FieldType(TextField.TYPE_UNSTORED);
+    customType.setStoreTermVectors(true);
+    customType.setStoreTermVectorPositions(true);
+    customType.setStoreTermVectorOffsets(true);
+    doc.add(newField("a", "a b a c a d a e a f a g a h a", customType));
+    doc.add(newField("b", "a b c b d b e b f b g b h b", customType));
+    doc.add(newField("c", "a c b c d c e c f c g c h c", customType));
     iw.addDocument(doc);
 
     doc = new Document();
-    doc.add(newField("a", "a b a c a d a e a f a g a h a", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS));
-    doc.add(newField("b", "a b c b d b e b f b g b h b", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS));
-    doc.add(newField("c", "a c b c d c e c f c g c h c", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS));
+    FieldType customType2 = new FieldType(TextField.TYPE_UNSTORED);
+    customType2.setStoreTermVectors(true);
+    customType2.setStoreTermVectorPositions(true);
+    doc.add(newField("a", "a b a c a d a e a f a g a h a", customType2));
+    doc.add(newField("b", "a b c b d b e b f b g b h b", customType2));
+    doc.add(newField("c", "a c b c d c e c f c g c h c", customType2));
     iw.addDocument(doc);
 
     doc = new Document();
-    doc.add(newField("a", "a b a c a d a e a f a g a h a", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.YES));
-    doc.add(newField("b", "a b c b d b e b f b g b h b", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.YES));
-    doc.add(newField("c", "a c b c d c e c f c g c h c", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.YES));
+    FieldType customType3 = new FieldType(TextField.TYPE_UNSTORED);
+    customType3.setStoreTermVectors(true);
+    doc.add(newField("a", "a b a c a d a e a f a g a h a", customType3));
+    doc.add(newField("b", "a b c b d b e b f b g b h b", customType3));
+    doc.add(newField("c", "a c b c d c e c f c g c h c", customType3));
     iw.addDocument(doc);
 
     doc = new Document();
-    doc.add(newField("a", "a b a c a d a e a f a g a h a", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.NO));
-    doc.add(newField("b", "a b c b d b e b f b g b h b", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.NO));
-    doc.add(newField("c", "a c b c d c e c f c g c h c", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.NO));
+    doc.add(newField("a", "a b a c a d a e a f a g a h a", TextField.TYPE_UNSTORED));
+    doc.add(newField("b", "a b c b d b e b f b g b h b", TextField.TYPE_UNSTORED));
+    doc.add(newField("c", "a c b c d c e c f c g c h c", TextField.TYPE_UNSTORED));
     iw.addDocument(doc);
 
     doc = new Document();
-    doc.add(newField("a", "a b a c a d a e a f a g a h a", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
-    doc.add(newField("b", "a b c b d b e b f b g b h b", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.NO));
-    doc.add(newField("c", "a c b c d c e c f c g c h c", Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.YES));
+    doc.add(newField("a", "a b a c a d a e a f a g a h a", customType));
+    doc.add(newField("b", "a b c b d b e b f b g b h b", TextField.TYPE_UNSTORED));
+    doc.add(newField("c", "a c b c d c e c f c g c h c", customType3));
     iw.addDocument(doc);
 
     iw.close();

Modified: lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java?rev=1162347&r1=1162346&r2=1162347&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/index/codecs/appending/TestAppendingCodec.java Sat Aug 27 13:27:01 2011
@@ -22,9 +22,8 @@ import java.util.Random;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Index;
-import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.Field.TermVector;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
@@ -141,7 +140,11 @@ public class TestAppendingCodec extends 
     ((TieredMergePolicy)cfg.getMergePolicy()).setUseCompoundFile(false);
     IndexWriter writer = new IndexWriter(dir, cfg);
     Document doc = new Document();
-    doc.add(newField("f", text, Store.YES, Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
+    FieldType storedTextType = new FieldType(TextField.TYPE_STORED);
+    storedTextType.setStoreTermVectors(true);
+    storedTextType.setStoreTermVectorPositions(true);
+    storedTextType.setStoreTermVectorOffsets(true);
+    doc.add(newField("f", text, storedTextType));
     writer.addDocument(doc);
     writer.commit();
     writer.addDocument(doc);
@@ -149,8 +152,8 @@ public class TestAppendingCodec extends 
     writer.close();
     IndexReader reader = IndexReader.open(dir, null, true, 1, new AppendingCodecProvider());
     assertEquals(2, reader.numDocs());
-    doc = reader.document(0);
-    assertEquals(text, doc.get("f"));
+    Document doc2 = reader.document(0);
+    assertEquals(text, doc2.get("f"));
     Fields fields = MultiFields.getFields(reader);
     Terms terms = fields.terms("f");
     assertNotNull(terms);