You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/08/16 00:03:52 UTC

svn commit: r1158029 [2/15] - in /lucene/dev/branches/fieldtype_conflicted: lucene/ lucene/contrib/ lucene/contrib/demo/src/java/org/apache/lucene/demo/ lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/ lucene/contrib/highlighter/...

Modified: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexWriter.java?rev=1158029&r1=1158028&r2=1158029&view=diff
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexWriter.java (original)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedIndexWriter.java Mon Aug 15 22:03:41 2011
@@ -37,9 +37,9 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermVectorOffsetInfo;
 import org.apache.lucene.search.IndexSearcher;
@@ -238,7 +238,7 @@ public class InstantiatedIndexWriter imp
         if (eFieldTermDocInfoFactoriesByTermText.getKey().indexed && !eFieldTermDocInfoFactoriesByTermText.getKey().omitNorms) {
           final String fieldName = eFieldTermDocInfoFactoriesByTermText.getKey().fieldName;
           final FieldInvertState invertState = new FieldInvertState();
-          invertState.setBoost(eFieldTermDocInfoFactoriesByTermText.getKey().boost * document.getDocument().getBoost());
+          invertState.setBoost(eFieldTermDocInfoFactoriesByTermText.getKey().boost);
           invertState.setLength(eFieldTermDocInfoFactoriesByTermText.getKey().fieldLength);
           normsByFieldNameAndDocumentNumber.get(fieldName)[document.getDocumentNumber()] = similarityProvider.get(fieldName).computeNorm(invertState);
         } else {
@@ -469,7 +469,7 @@ public class InstantiatedIndexWriter imp
     // normalize settings per field name in document
 
     Map<String /* field name */, FieldSetting> fieldSettingsByFieldName = new HashMap<String, FieldSetting>();
-    for (Fieldable field : document.getDocument().getFields()) {
+    for (IndexableField field : document.getDocument()) {
       FieldSetting fieldSetting = fieldSettingsByFieldName.get(field.name());
       if (fieldSetting == null) {
         fieldSetting = new FieldSetting();
@@ -479,52 +479,52 @@ public class InstantiatedIndexWriter imp
       }
 
       // todo: fixme: multiple fields with the same name does not mean field boost += more boost.
-      fieldSetting.boost *= field.getBoost();
+      fieldSetting.boost *= field.boost();
       //fieldSettings.dimensions++;
 
 
       // once fieldSettings, always fieldSettings.
-      if (field.getOmitNorms()) {
+      if (field.omitNorms()) {
         fieldSetting.omitNorms = true;
       }
-      if (field.isIndexed() ) {
+      if (field.indexed() ) {
         fieldSetting.indexed = true;
       }
-      if (field.isTokenized()) {
+      if (field.tokenized()) {
         fieldSetting.tokenized = true;
       }
-      if (field.isStored()) {
+      if (field.stored()) {
         fieldSetting.stored = true;
       }
-      if (field.isBinary()) {
+      if (field.binaryValue(null) != null) {
         fieldSetting.isBinary = true;
       }
-      if (field.isTermVectorStored()) {
+      if (field.storeTermVectors()) {
         fieldSetting.storeTermVector = true;
       }
-      if (field.isStorePositionWithTermVector()) {
+      if (field.storeTermVectorPositions()) {
         fieldSetting.storePositionWithTermVector = true;
       }
-      if (field.isStoreOffsetWithTermVector()) {
+      if (field.storeTermVectorOffsets()) {
         fieldSetting.storeOffsetWithTermVector = true;
       }
     }
 
-    Map<Fieldable, LinkedList<Token>> tokensByField = new LinkedHashMap<Fieldable, LinkedList<Token>>(20);
+    Map<IndexableField, LinkedList<Token>> tokensByField = new LinkedHashMap<IndexableField, LinkedList<Token>>(20);
 
     // tokenize indexed fields.
-    for (Iterator<Fieldable> it = document.getDocument().getFields().iterator(); it.hasNext();) {
+    for (Iterator<IndexableField> it = document.getDocument().iterator(); it.hasNext();) {
 
-      Fieldable field = it.next();
+      IndexableField field = it.next();
 
       FieldSetting fieldSetting = fieldSettingsByFieldName.get(field.name());
 
-      if (field.isIndexed()) {
+      if (field.indexed()) {
 
         LinkedList<Token> tokens = new LinkedList<Token>();
         tokensByField.put(field, tokens);
 
-        if (field.isTokenized()) {
+        if (field.tokenized()) {
           final TokenStream tokenStream;
           // todo readerValue(), binaryValue()
           if (field.tokenStreamValue() != null) {
@@ -564,8 +564,8 @@ public class InstantiatedIndexWriter imp
         }
       }
 
-      if (!field.isStored()) {
-        it.remove();
+      if (!field.stored()) {
+        //it.remove();
       }
     }
 
@@ -574,7 +574,7 @@ public class InstantiatedIndexWriter imp
     termDocumentInformationFactoryByDocument.put(document, termDocumentInformationFactoryByTermTextAndFieldSetting);
 
     // build term vector, term positions and term offsets
-    for (Map.Entry<Fieldable, LinkedList<Token>> eField_Tokens : tokensByField.entrySet()) {
+    for (Map.Entry<IndexableField, LinkedList<Token>> eField_Tokens : tokensByField.entrySet()) {
       FieldSetting fieldSetting = fieldSettingsByFieldName.get(eField_Tokens.getKey().name());
 
       Map<String, TermDocumentInformationFactory> termDocumentInformationFactoryByTermText = termDocumentInformationFactoryByTermTextAndFieldSetting.get(fieldSettingsByFieldName.get(eField_Tokens.getKey().name()));
@@ -610,7 +610,7 @@ public class InstantiatedIndexWriter imp
           termDocumentInformationFactory.payloads.add(null);
         }
 
-        if (eField_Tokens.getKey().isStoreOffsetWithTermVector()) {
+        if (eField_Tokens.getKey().storeTermVectorOffsets()) {
 
           termDocumentInformationFactory.termOffsets.add(new TermVectorOffsetInfo(fieldSetting.offset + token.startOffset(), fieldSetting.offset + token.endOffset()));
           lastOffset = fieldSetting.offset + token.endOffset();
@@ -619,7 +619,7 @@ public class InstantiatedIndexWriter imp
 
       }
 
-      if (eField_Tokens.getKey().isStoreOffsetWithTermVector()) {
+      if (eField_Tokens.getKey().storeTermVectorOffsets()) {
         fieldSetting.offset = lastOffset + 1;
       }
 

Modified: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java?rev=1158029&r1=1158028&r2=1158029&view=diff
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java (original)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestIndicesEquals.java Mon Aug 15 22:03:41 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,47 @@ 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_UNSTORED);
+    customType.setStored(true);
+    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_UNSTORED);
+      customType2.setStored(true);
+      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_UNSTORED);
+      customType4.setStored(true);
+      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 +277,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/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestRealTime.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestRealTime.java?rev=1158029&r1=1158028&r2=1158029&view=diff
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestRealTime.java (original)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestRealTime.java Mon Aug 15 22:03:41 2011
@@ -20,6 +20,7 @@ import org.apache.lucene.search.TermQuer
 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 +44,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 +53,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/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestUnoptimizedReaderOnConstructor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestUnoptimizedReaderOnConstructor.java?rev=1158029&r1=1158028&r2=1158029&view=diff
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestUnoptimizedReaderOnConstructor.java (original)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/TestUnoptimizedReaderOnConstructor.java Mon Aug 15 22:03:41 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/branches/fieldtype_conflicted/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1158029&r1=1158028&r2=1158029&view=diff
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Mon Aug 15 22:03:41 2011
@@ -35,23 +35,28 @@ 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 +65,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 +245,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 +1239,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/branches/fieldtype_conflicted/lucene/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java?rev=1158029&r1=1158028&r2=1158029&view=diff
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java (original)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java Mon Aug 15 22:03:41 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);

Added: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelector.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelector.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelector.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,36 @@
+package org.apache.lucene.document;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+
+/**
+ * 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.
+ */
+
+/**
+ * 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 FieldSelectorVisitor}
+ *
+ **/
+public interface FieldSelector {
+
+  /**
+   * 
+   * @param fieldName the field to accept or reject
+   * @return an instance of {@link FieldSelectorResult}
+   * if the {@link Field} named <code>fieldName</code> should be loaded.
+   */
+  FieldSelectorResult accept(String fieldName);
+}

Added: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorResult.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorResult.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorResult.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorResult.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,78 @@
+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.
+ */
+
+/**
+ *  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)} should not return null.
+     *<p/>
+     * {@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)} is safe to use and should
+     * return a valid instance of a {@link IndexableField}.
+     *<p/>
+     * {@link Document#add(IndexableField)} should be called by the Reader.
+     */
+  LAZY_LOAD,
+
+    /**
+     * Do not load the {@link Field}.  {@link Document#getField(String)} should return null.
+     * {@link Document#add(IndexableField)} is not called.
+     * <p/>
+     * {@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)} should
+     * both be valid for this {@link Field}
+     * <p/>
+     * {@link Document#add(IndexableField)} should be called by the Reader.
+     */
+  LOAD_AND_BREAK,
+
+    /** Expert:  Load the size of this {@link Field} rather than its value.
+     * Size is measured as number of bytes required to store the field == bytes for a binary or any compressed value, and 2*chars for a String value.
+     * The size is stored as a binary value, represented as an int in a byte[], with the higher order byte first in [0]
+     */
+  SIZE,
+
+    /** Expert: Like {@link #SIZE} but immediately break from the field loading loop, i.e., stop loading further fields, after the size is loaded */         
+  SIZE_AND_BREAK,
+
+  /**
+     * 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)} is safe to use and should
+     * return a valid instance of a {@link IndexableField}.
+     *<p/>
+     * {@link Document#add(IndexableField)} should be called by the Reader.
+     */
+  LATENT
+}

Added: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorVisitor.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorVisitor.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/FieldSelectorVisitor.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,335 @@
+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.BinaryField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.NumericField;
+import org.apache.lucene.document.TextField;
+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.setOmitTermFreqAndPositions(fieldInfo.omitTermFreqAndPositions);
+    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.setOmitTermFreqAndPositions(fieldInfo.omitTermFreqAndPositions);
+    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.setOmitTermFreqAndPositions(fieldInfo.omitTermFreqAndPositions);
+    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.setOmitTermFreqAndPositions(fieldInfo.omitTermFreqAndPositions);
+    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.setOmitTermFreqAndPositions(fi.omitTermFreqAndPositions);
+    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;
+
+    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;
+      if (isBinary)
+        binaryLength = toRead;
+    }
+
+    @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. */
+    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. */
+    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. */
+    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;
+        }
+      }
+    }
+
+    synchronized private byte[] getBinaryValue(byte[] result) {
+      if (isBinary) {
+        if (fieldsData == null) {
+          // Allocate new buffer if result is null or too small
+          final byte[] b;
+          if (result == null || result.length < toRead)
+            b = new byte[toRead];
+          else
+            b = result;
+   
+          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);
+          }
+
+          binaryOffset = 0;
+          binaryLength = toRead;
+          if (cacheResult == true){
+            fieldsData = b;
+          }
+          return b;
+        } else {
+          return (byte[]) fieldsData;
+        }
+      } else
+        return null;     
+    }
+
+    @Override
+    public BytesRef binaryValue(BytesRef reuse) {
+      final byte[] bytes = getBinaryValue(reuse != null ? reuse.bytes : null);
+      if (bytes != null) {
+        return new BytesRef(bytes, 0, bytes.length);
+      } else {
+        return null;
+      }
+    }
+  }
+}
\ No newline at end of file

Added: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/LoadFirstFieldSelector.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,29 @@
+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.
+ */
+
+
+/**
+ * Load the First field and break.
+ * <p/>
+ * See {@link FieldSelectorResult#LOAD_AND_BREAK}
+ */
+public class LoadFirstFieldSelector implements FieldSelector {
+
+  public FieldSelectorResult accept(String fieldName) {
+    return FieldSelectorResult.LOAD_AND_BREAK;
+  }
+}
\ No newline at end of file

Added: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/MapFieldSelector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/MapFieldSelector.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/MapFieldSelector.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/MapFieldSelector.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,67 @@
+package org.apache.lucene.document;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link FieldSelector} based on a Map of field names to {@link FieldSelectorResult}s
+ *
+ */
+public class MapFieldSelector implements FieldSelector {
+    
+    Map<String,FieldSelectorResult> fieldSelections;
+    
+    /** Create a a MapFieldSelector
+     * @param fieldSelections maps from field names (String) to {@link FieldSelectorResult}s
+     */
+    public MapFieldSelector(Map<String,FieldSelectorResult> fieldSelections) {
+        this.fieldSelections = fieldSelections;
+    }
+    
+    /** Create a a MapFieldSelector
+     * @param fields fields to LOAD.  List of Strings.  All other fields are NO_LOAD.
+     */
+    public MapFieldSelector(List<String> fields) {
+        fieldSelections = new HashMap<String,FieldSelectorResult>(fields.size()*5/3);
+        for (final String field : fields)
+            fieldSelections.put(field, FieldSelectorResult.LOAD);
+    }
+    
+    /** Create a a MapFieldSelector
+     * @param fields fields to LOAD.  All other fields are NO_LOAD.
+     */
+    public MapFieldSelector(String... fields) {
+      this(Arrays.asList(fields));
+    }
+
+
+    
+    /** Load field according to its associated value in fieldSelections
+     * @param field a field name
+     * @return the fieldSelections value that field maps to or NO_LOAD if none.
+     */
+    public FieldSelectorResult accept(String field) {
+        FieldSelectorResult selection = fieldSelections.get(field);
+        return selection!=null ? selection : FieldSelectorResult.NO_LOAD;
+    }
+    
+}

Added: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/SetBasedFieldSelector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/SetBasedFieldSelector.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/SetBasedFieldSelector.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/document/SetBasedFieldSelector.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,62 @@
+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.util.Set;
+
+import org.apache.lucene.document.Field;
+
+/**
+ * Declare what fields to load normally and what fields to load lazily
+ *
+ **/
+
+public class SetBasedFieldSelector implements FieldSelector {
+  
+  private Set<String> fieldsToLoad;
+  private Set<String> lazyFieldsToLoad;
+  
+  /**
+   * Pass in the Set of {@link Field} names to load and the Set of {@link Field} names to load lazily.  If both are null, the
+   * Document will not have any {@link Field} on it.  
+   * @param fieldsToLoad A Set of {@link String} field names to load.  May be empty, but not null
+   * @param lazyFieldsToLoad A Set of {@link String} field names to load lazily.  May be empty, but not null  
+   */
+  public SetBasedFieldSelector(Set<String> fieldsToLoad, Set<String> lazyFieldsToLoad) {
+    this.fieldsToLoad = fieldsToLoad;
+    this.lazyFieldsToLoad = lazyFieldsToLoad;
+  }
+
+  /**
+   * Indicate whether to load the field with the given name or not. If the {@link Field#name()} is not in either of the 
+   * initializing Sets, then {@link org.apache.lucene.document.FieldSelectorResult#NO_LOAD} is returned.  If a Field name
+   * is in both <code>fieldsToLoad</code> and <code>lazyFieldsToLoad</code>, lazy has precedence.
+   * 
+   * @param fieldName The {@link Field} name to check
+   * @return The {@link FieldSelectorResult}
+   */
+  public FieldSelectorResult accept(String fieldName) {
+    FieldSelectorResult result = FieldSelectorResult.NO_LOAD;
+    if (fieldsToLoad.contains(fieldName) == true){
+      result = FieldSelectorResult.LOAD;
+    }
+    if (lazyFieldsToLoad.contains(fieldName) == true){
+      result = FieldSelectorResult.LAZY_LOAD;
+    }                                           
+    return result;
+  }
+}
\ No newline at end of file

Modified: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java?rev=1158029&r1=1158028&r2=1158029&view=diff
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java (original)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java Mon Aug 15 22:03:41 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/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribFieldsReader.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribFieldsReader.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribFieldsReader.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,319 @@
+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.AlreadyClosedException;
+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) field).lazy());
+    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);
+    assertTrue("Field is lazy and it should not be", ((Field) field).lazy() == false);
+    field = doc.getField(DocHelper.TEXT_FIELD_UTF1_KEY);
+    assertTrue("field is null and it shouldn't be", field != null);
+    assertTrue("Field is lazy and it should not be", ((Field) field).lazy() == false);
+    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) field).lazy() == true);
+    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(null).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(null).bytes == field.binaryValue(null).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) field).lazy());
+    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);
+    assertTrue("Field is lazy and it should not be", ((Field) field).lazy() == false);
+    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);
+    assertTrue("Field is lazy and it should not be", ((Field) field).lazy() == false);
+    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) field).lazy() == true);
+    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(null).bytes != field.binaryValue(null).bytes);
+
+    byte [] bytes = field.binaryValue(null).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);
+      assertTrue("field is lazy", ((Field) field).lazy() == false);
+      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) field).lazy() == true);
+      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)!=null);
+    assertTrue(f3.binaryValue(null)==null);
+    assertTrue(fb.binaryValue(null)!=null);
+    assertSizeEquals(2*DocHelper.FIELD_1_TEXT.length(), f1.binaryValue(null).bytes);
+    assertEquals(DocHelper.FIELD_3_TEXT, f3.stringValue());
+    assertSizeEquals(DocHelper.LAZY_FIELD_BINARY_BYTES.length, fb.binaryValue(null).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]);
+  }
+}
\ No newline at end of file

Added: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribIndexReader.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribIndexReader.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribIndexReader.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,188 @@
+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.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 customType = new FieldType(TextField.TYPE_UNSTORED);
+    customType.setStored(true);
+    customType.setTokenized(false);
+
+    FieldType customType2 = new FieldType(TextField.TYPE_UNSTORED);
+    customType2.setStored(true);
+
+    FieldType customType3 = new FieldType();
+    customType3.setStored(true);
+    doc.add(newField("keyword", "test1", customType));
+    doc.add(newField("text", "test1", customType2));
+    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 customType = new FieldType(TextField.TYPE_UNSTORED);
+    customType.setStored(true);
+    customType.setTokenized(false);
+
+    FieldType customType2 = new FieldType(TextField.TYPE_UNSTORED);
+    customType2.setStored(true);
+
+    FieldType customType3 = new FieldType();
+    customType3.setStored(true);
+    doc.add(newField("keyword2", "test1", customType));
+    doc.add(newField("text2", "test1", customType2));
+    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 customType4 = new FieldType(TextField.TYPE_UNSTORED);
+    customType4.setStored(true);
+    FieldType customType5 = new FieldType(TextField.TYPE_UNSTORED);
+    customType5.setStored(true);
+    customType5.setStoreTermVectors(true);
+    FieldType customType6 = new FieldType(TextField.TYPE_UNSTORED);
+    customType6.setStored(true);
+    customType6.setStoreTermVectors(true);
+    customType6.setStoreTermVectorOffsets(true);
+    FieldType customType7 = new FieldType(TextField.TYPE_UNSTORED);
+    customType7.setStored(true);
+    customType7.setStoreTermVectors(true);
+    customType7.setStoreTermVectorPositions(true);
+    FieldType customType8 = new FieldType(TextField.TYPE_UNSTORED);
+    customType8.setStored(true);
+    customType8.setStoreTermVectors(true);
+    customType8.setStoreTermVectorOffsets(true);
+    customType8.setStoreTermVectorPositions(true);
+    doc.add(newField("tvnot","tvnot",customType4));
+    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(null);
+    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)!=null);
+    bytesRef = fb1.binaryValue(null);
+    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(null);
+    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/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribParallelReader.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribParallelReader.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestContribParallelReader.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,156 @@
+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.FieldType;
+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();
+    FieldType customType = new FieldType(TextField.TYPE_UNSTORED);
+    customType.setStored(true);
+    d1.add(newField("f1", "v1", customType));
+    d1.add(newField("f2", "v1", customType));
+    d1.add(newField("f3", "v1", customType));
+    d1.add(newField("f4", "v1", customType));
+    w.addDocument(d1);
+    Document d2 = new Document();
+    d2.add(newField("f1", "v2", customType));
+    d2.add(newField("f2", "v2", customType));
+    d2.add(newField("f3", "v2", customType));
+    d2.add(newField("f4", "v2", customType));
+    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();
+    FieldType customType = new FieldType(TextField.TYPE_UNSTORED);
+    customType.setStored(true);
+    d1.add(newField("f1", "v1", customType));
+    d1.add(newField("f2", "v1", customType));
+    w1.addDocument(d1);
+    Document d2 = new Document();
+    d2.add(newField("f1", "v2", customType));
+    d2.add(newField("f2", "v2", customType));
+    w1.addDocument(d2);
+    w1.close();
+    return dir1;
+  }
+
+  private Directory getDir2(Random random) throws IOException {
+    Directory dir2 = newDirectory();
+    FieldType customType = new FieldType(TextField.TYPE_UNSTORED);
+    customType.setStored(true);
+    IndexWriter w2 = new IndexWriter(dir2, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)));
+    Document d3 = new Document();
+    d3.add(newField("f3", "v1", customType));
+    d3.add(newField("f4", "v1", customType));
+    w2.addDocument(d3);
+    Document d4 = new Document();
+    d4.add(newField("f3", "v2", customType));
+    d4.add(newField("f4", "v2", customType));
+    w2.addDocument(d4);
+    w2.close();
+    return dir2;
+  }
+}

Modified: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java?rev=1158029&r1=1158028&r2=1158029&view=diff
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java (original)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java Mon Aug 15 22:03:41 2011
@@ -23,6 +23,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.FieldType;
+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 +67,21 @@ 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));
+      
+      FieldType storedTextType = new FieldType(TextField.TYPE_UNSTORED);
+      storedTextType.setStored(true);
+      d.add(newField("field", "word", storedTextType));
+
+      FieldType storedTextType2 = new FieldType(TextField.TYPE_UNSTORED);
+      storedTextType2.setStored(true);
+      storedTextType2.setTokenized(false);
+      storedTextType2.setOmitNorms(true);
+      d.add(newField("nonorm", "word", storedTextType2));
+      d.add(newField("untokfield", "20061212 20071212", storedTextType));
       
       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", storedTextType));
+        d.add(newField("nonorm", "more words", storedTextType2));
       }
       writer.addDocument(d);
     }

Added: lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestLazyBug.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestLazyBug.java?rev=1158029&view=auto
==============================================================================
--- lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestLazyBug.java (added)
+++ lucene/dev/branches/fieldtype_conflicted/lucene/contrib/misc/src/test/org/apache/lucene/index/TestLazyBug.java Mon Aug 15 22:03:41 2011
@@ -0,0 +1,145 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+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;
+import org.junit.BeforeClass;
+
+
+/**
+ * Test demonstrating EOF bug on the last field of the last doc
+ * if other docs have allready been accessed.
+ */
+public class TestLazyBug extends LuceneTestCase {
+
+  public static int NUM_DOCS = TEST_NIGHTLY ? 500 : 50;
+  public static int NUM_FIELDS = TEST_NIGHTLY ? 100 : 10;
+
+  private static String[] data = new String[] {
+    "now",
+    "is the time",
+    "for all good men",
+    "to come to the aid",
+    "of their country!",
+    "this string contains big chars:{\u0111 \u0222 \u0333 \u1111 \u2222 \u3333}",
+    "this string is a bigger string, mary had a little lamb, little lamb, little lamb!"
+  };
+
+  private static Set<String> dataset = asSet(data);
+
+  private static String MAGIC_FIELD = "f"+(NUM_FIELDS/3);
+  
+  private static Directory directory;
+  
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    directory = makeIndex();
+  }
+  
+  @AfterClass
+  public static void afterClass() throws Exception {
+    directory.close();
+    directory = null;
+  }
+
+  private static FieldSelector SELECTOR = new FieldSelector() {
+      public FieldSelectorResult accept(String f) {
+        if (f.equals(MAGIC_FIELD)) {
+          return FieldSelectorResult.LOAD;
+        }
+        return FieldSelectorResult.LAZY_LOAD;
+      }
+    };
+
+  private static Directory makeIndex() throws Exception {
+    Directory dir = newDirectory();
+    try {
+      IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
+                                                                     TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
+      LogMergePolicy lmp = (LogMergePolicy) writer.getConfig().getMergePolicy();
+      lmp.setUseCompoundFile(false);
+      for (int d = 1; d <= NUM_DOCS; d++) {
+        Document doc = new Document();
+        for (int f = 1; f <= NUM_FIELDS; f++ ) {
+          doc.add(newField("f"+f,
+                            data[f % data.length]
+                            + '#' + data[random.nextInt(data.length)],
+                            TextField.TYPE_UNSTORED));
+        }
+        writer.addDocument(doc);
+      }
+      writer.close();
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return dir;
+  }
+
+  public void doTest(int[] docs) throws Exception {
+    IndexReader reader = IndexReader.open(directory, true);
+    for (int i = 0; i < docs.length; i++) {
+      final FieldSelectorVisitor visitor = new FieldSelectorVisitor(SELECTOR);
+      reader.document(docs[i], visitor);
+      Document d = visitor.getDocument();
+      d.get(MAGIC_FIELD);
+
+      List<IndexableField> fields = d.getFields();
+      for (Iterator<IndexableField> fi = fields.iterator(); fi.hasNext(); ) {
+        IndexableField f=null;
+        try {
+          f =  fi.next();
+          String fname = f.name();
+          String fval = f.stringValue();
+          assertNotNull(docs[i]+" FIELD: "+fname, fval);
+          String[] vals = fval.split("#");
+          if (!dataset.contains(vals[0]) || !dataset.contains(vals[1])) {
+            fail("FIELD:"+fname+",VAL:"+fval);
+          }
+        } catch (Exception e) {
+          throw new Exception(docs[i]+" WTF: "+f.name(), e);
+        }
+      }
+    }
+    reader.close();
+  }
+
+  public void testLazyWorks() throws Exception {
+    doTest(new int[] { NUM_DOCS-1 });
+  }
+
+  public void testLazyAlsoWorks() throws Exception {
+    doTest(new int[] { NUM_DOCS-1, NUM_DOCS/2 });
+  }
+
+  public void testLazyBroken() throws Exception {
+    doTest(new int[] { NUM_DOCS/2, NUM_DOCS-1 });
+  }
+
+}