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/30 01:13:23 UTC

svn commit: r1163047 [5/15] - in /lucene/dev/branches/flexscoring: ./ dev-tools/idea/lucene/contrib/ lucene/ lucene/contrib/ lucene/contrib/demo/src/java/org/apache/lucene/demo/ lucene/contrib/demo/src/java/org/apache/lucene/demo/xmlparser/ lucene/cont...

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriter.java Mon Aug 29 23:13:10 2011
@@ -150,7 +150,7 @@ public final class SepPostingsWriter ext
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeSafely(true, docOut, skipOut, freqOut, posOut, payloadOut);
+        IOUtils.closeWhileHandlingException(docOut, skipOut, freqOut, posOut, payloadOut);
       }
 
     }
@@ -389,7 +389,7 @@ public final class SepPostingsWriter ext
 
   @Override
   public void close() throws IOException {
-    IOUtils.closeSafely(false, docOut, skipOut, freqOut, posOut, payloadOut);
+    IOUtils.close(docOut, skipOut, freqOut, posOut, payloadOut);
   }
 
   public static void getExtensions(Set<String> extensions) {

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/Bytes.java Mon Aug 29 23:13:10 2011
@@ -368,7 +368,7 @@ public final class Bytes {
           success = true;
         } finally {
           if (!success) {
-            IOUtils.closeSafely(true, datOut);
+            IOUtils.closeWhileHandlingException(datOut);
           }
         }
       }
@@ -386,7 +386,7 @@ public final class Bytes {
         success = true;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, idxOut);
+          IOUtils.closeWhileHandlingException(idxOut);
         }
       }
       return idxOut;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Mon Aug 29 23:13:10 2011
@@ -112,7 +112,11 @@ class FixedDerefBytesImpl {
         }
         success = true;
       } finally {
-        IOUtils.closeSafely(!success, datOut);
+        if (success) {
+          IOUtils.close(datOut);
+        } else {
+          IOUtils.closeWhileHandlingException(datOut);
+        }
         hash.close();
       }
       success = false;
@@ -134,7 +138,11 @@ class FixedDerefBytesImpl {
         w.finish();
         success = true;
       } finally {
-        IOUtils.closeSafely(!success, idxOut);
+        if (success) {
+          IOUtils.close(idxOut);
+        } else {
+          IOUtils.closeWhileHandlingException(idxOut);
+        }
         bytesUsed
             .addAndGet((-docToID.length) * RamUsageEstimator.NUM_BYTES_INT);
         docToID = null;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Mon Aug 29 23:13:10 2011
@@ -126,7 +126,11 @@ class FixedSortedBytesImpl {
         }
         success = true;
       } finally {
-        IOUtils.closeSafely(!success, datOut);
+        if (success) {
+          IOUtils.close(datOut);
+        } else {
+          IOUtils.closeWhileHandlingException(datOut);
+        }
         hash.close();
       }
       final IndexOutput idxOut = getIndexOut();
@@ -159,7 +163,11 @@ class FixedSortedBytesImpl {
         }
         w.finish();
       } finally {
-        IOUtils.closeSafely(!success, idxOut);
+        if (success) {
+          IOUtils.close(idxOut);
+        } else {
+          IOUtils.closeWhileHandlingException(idxOut);
+        }
         bytesUsed.addAndGet((-docToEntry.length)
             * RamUsageEstimator.NUM_BYTES_INT);
         docToEntry = null;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Mon Aug 29 23:13:10 2011
@@ -104,41 +104,39 @@ class FixedStraightBytesImpl {
       datOut = getDataOut();
       boolean success = false;
       try {
-      if (state.liveDocs == null && state.reader instanceof Reader) {
-        Reader reader = (Reader) state.reader;
-        final int maxDocs = reader.maxDoc;
-        if (maxDocs == 0) {
-          return;
-        }
-        if (size == -1) {
-          size = reader.size;
-          datOut.writeInt(size);
-        }
-        if (lastDocID+1 < state.docBase) {
-          fill(datOut, state.docBase);
-          lastDocID = state.docBase-1;
-        }
-        // TODO should we add a transfer to API to each reader?
-        final IndexInput cloneData = reader.cloneData();
-        try {
-          datOut.copyBytes(cloneData, size * maxDocs);
-        } finally {
-          IOUtils.closeSafely(true, cloneData);  
-        }
+        if (state.liveDocs == null && state.reader instanceof Reader) {
+          Reader reader = (Reader) state.reader;
+          final int maxDocs = reader.maxDoc;
+          if (maxDocs == 0) {
+            return;
+          }
+          if (size == -1) {
+            size = reader.size;
+            datOut.writeInt(size);
+          }
+          if (lastDocID+1 < state.docBase) {
+            fill(datOut, state.docBase);
+            lastDocID = state.docBase-1;
+          }
+          // TODO should we add a transfer to API to each reader?
+          final IndexInput cloneData = reader.cloneData();
+          try {
+            datOut.copyBytes(cloneData, size * maxDocs);
+          } finally {
+            IOUtils.close(cloneData);  
+          }
         
-        lastDocID += maxDocs;
-      } else {
-        super.merge(state);
-      }
-      success = true;
+          lastDocID += maxDocs;
+        } else {
+          super.merge(state);
+        }
+        success = true;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(!success, datOut);
+          IOUtils.closeWhileHandlingException(datOut);
         }
       }
     }
-    
-    
 
     @Override
     protected void mergeDoc(int docID) throws IOException {
@@ -196,7 +194,11 @@ class FixedStraightBytesImpl {
         success = true;
       } finally {
         pool.dropBuffersAndReset();
-        IOUtils.closeSafely(!success, datOut);
+        if (success) {
+          IOUtils.close(datOut);
+        } else {
+          IOUtils.closeWhileHandlingException(datOut);
+        }
       }
     }
   }
@@ -233,7 +235,7 @@ class FixedStraightBytesImpl {
           data = new byte[maxDoc];
           datIn.readBytes(data, 0, data.length, false);
         } finally {
-          IOUtils.closeSafely(false, datIn);
+          IOUtils.close(datIn);
         }
 
       }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/Floats.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/Floats.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/Floats.java Mon Aug 29 23:13:10 2011
@@ -102,7 +102,7 @@ public class Floats {
         success = true;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, datOut);
+          IOUtils.closeWhileHandlingException(datOut);
         }
       }
     }
@@ -214,11 +214,13 @@ public class Floats {
         bytesUsed.addAndGet(-(RamUsageEstimator.NUM_BYTES_INT
             * ((values.length))));
         values = null;
-        IOUtils.closeSafely(!success, datOut);
+        if (success) {
+          IOUtils.close(datOut);
+        } else {
+          IOUtils.closeWhileHandlingException(datOut);
+        }
       }
     }
-
-    
   }
 
   // Writes 8 bytes (double) per value
@@ -275,7 +277,11 @@ public class Floats {
         bytesUsed.addAndGet(-(RamUsageEstimator.NUM_BYTES_LONG
             * ((values.length))));
         values = null;
-        IOUtils.closeSafely(!success, datOut);
+        if (success) {
+          IOUtils.close(datOut);
+        } else {
+          IOUtils.closeWhileHandlingException(datOut);
+        }
       }
     }
   }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/IntsImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/IntsImpl.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/IntsImpl.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/IntsImpl.java Mon Aug 29 23:13:10 2011
@@ -21,7 +21,6 @@ import java.util.Collection;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.values.IndexDocValuesArray;
 import org.apache.lucene.index.values.IndexDocValues.Source;
 import org.apache.lucene.index.values.IndexDocValues.SourceEnum;
 import org.apache.lucene.index.values.IndexDocValuesArray.ByteValues;
@@ -132,7 +131,7 @@ class IntsImpl {
 
         } finally {
           if (!success) {
-            IOUtils.closeSafely(true, datOut);
+            IOUtils.closeWhileHandlingException(datOut);
           }
         }
       }
@@ -153,7 +152,11 @@ class IntsImpl {
         }
         success = true;
       } finally {
-        IOUtils.closeSafely(!success, datOut);
+        if (success) {
+          IOUtils.close(datOut);
+        } else {
+          IOUtils.closeWhileHandlingException(datOut);
+        }
         array.clear();
       }
     }
@@ -287,7 +290,7 @@ class IntsImpl {
         success = true;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, datIn);
+          IOUtils.closeWhileHandlingException(datIn);
         }
       }
     }
@@ -302,7 +305,11 @@ class IntsImpl {
         datOut.copyBytes(indexInput, bytesPerValue(type) * numDocs);
         success = true;
       } finally {
-        IOUtils.closeSafely(!success, indexInput);
+        if (success) {
+          IOUtils.close(indexInput);
+        } else {
+          IOUtils.closeWhileHandlingException(indexInput);
+        }
       }
       return numDocs;
     }
@@ -319,12 +326,12 @@ class IntsImpl {
       try {
         input = (IndexInput) datIn.clone();
         input.seek(CodecUtil.headerLength(CODEC_NAME) + 1);
-        source  = loadFixedSource(type, input, numDocs);
+        source = loadFixedSource(type, input, numDocs);
         success = true;
         return source;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, input, datIn);
+          IOUtils.closeWhileHandlingException(input, datIn);
         }
       }
     }
@@ -346,7 +353,7 @@ class IntsImpl {
         return inst;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, input);
+          IOUtils.closeWhileHandlingException(input);
         }
       }
     }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java Mon Aug 29 23:13:10 2011
@@ -19,14 +19,12 @@ package org.apache.lucene.index.values;
 import java.util.Comparator;
 
 import org.apache.lucene.document.IndexDocValuesField;
-import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.codecs.DocValuesConsumer;
 import org.apache.lucene.util.BytesRef;
 
 /**
  * Per document and field values consumed by {@link DocValuesConsumer}. 
  * @see IndexDocValuesField
- * @see Fieldable#setDocValues(PerDocFieldValues)
  * 
  * @lucene.experimental
  */
@@ -91,11 +89,10 @@ public interface PerDocFieldValues {
   /**
    * Sets the {@link ValueType}
    */
-  public void setType(ValueType type);
+  public void setDocValuesType(ValueType type);
 
   /**
   * Returns the {@link ValueType}
   */
-  public ValueType type();
-
-}
\ No newline at end of file
+  public ValueType docValuesType();
+}

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java Mon Aug 29 23:13:10 2011
@@ -184,7 +184,11 @@ class VarDerefBytesImpl {
         success = true;
       } finally {
         hash.close();
-        IOUtils.closeSafely(!success, datOut);
+        if (success) {
+          IOUtils.close(datOut);
+        } else {
+          IOUtils.closeWhileHandlingException(datOut);
+        }
       }
       
       final IndexOutput idxOut = getIndexOut();
@@ -211,7 +215,11 @@ class VarDerefBytesImpl {
         w.finish();
         success = true;
       } finally {
-        IOUtils.closeSafely(!success,idxOut);
+        if (success) {
+          IOUtils.close(idxOut);
+        } else {
+          IOUtils.closeWhileHandlingException(idxOut);
+        }
         bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
             * (-docToAddress.length));
         docToAddress = null;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java Mon Aug 29 23:13:10 2011
@@ -124,7 +124,11 @@ class VarSortedBytesImpl {
         }
         success = true;
       } finally {
-        IOUtils.closeSafely(!success, datOut);
+        if (success) {
+          IOUtils.close(datOut);
+        } else {
+          IOUtils.closeWhileHandlingException(datOut);
+        }
         hash.close();
       }
       final IndexOutput idxOut = getIndexOut();
@@ -161,7 +165,11 @@ class VarSortedBytesImpl {
         bytesUsed.addAndGet((-docToEntry.length)
             * RamUsageEstimator.NUM_BYTES_INT);
         docToEntry = null;
-        IOUtils.closeSafely(!success, idxOut);
+        if (success) {
+          IOUtils.close(idxOut);
+        } else {
+          IOUtils.closeWhileHandlingException(idxOut);
+        }
       }
     }
   }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java Mon Aug 29 23:13:10 2011
@@ -128,13 +128,13 @@ class VarStraightBytesImpl {
             address += numDataBytes; // this is the address after all addr pointers are updated
             iter.close();
           } finally {
-            IOUtils.closeSafely(true, cloneIdx);
+            IOUtils.close(cloneIdx);
           }
           final IndexInput cloneData = reader.cloneData();
           try {
             datOut.copyBytes(cloneData, numDataBytes);
           } finally {
-            IOUtils.closeSafely(true, cloneData);  
+            IOUtils.close(cloneData);  
           }
         } else {
           super.merge(state);
@@ -142,7 +142,7 @@ class VarStraightBytesImpl {
         success = true;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(!success, datOut);
+          IOUtils.closeWhileHandlingException(datOut);
         }
       }
     }
@@ -174,7 +174,11 @@ class VarStraightBytesImpl {
         }
         success = true;
       } finally {
-        IOUtils.closeSafely(!success, datOut); 
+        if (success) {
+          IOUtils.close(datOut);
+        } else {
+          IOUtils.closeWhileHandlingException(datOut);
+        }
         pool.dropBuffersAndReset();
       }
 
@@ -204,7 +208,11 @@ class VarStraightBytesImpl {
         bytesUsed.addAndGet(-(docToAddress.length)
             * RamUsageEstimator.NUM_BYTES_INT);
         docToAddress = null;
-        IOUtils.closeSafely(!success, idxOut);
+        if (success) {
+          IOUtils.close(idxOut);
+        } else {
+          IOUtils.closeWhileHandlingException(idxOut);
+        }
       }
     }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Mon Aug 29 23:13:10 2011
@@ -31,11 +31,11 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.IndexReader;
 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.StoredFieldVisitor;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Weight.ScorerContext;
 import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
@@ -241,16 +241,16 @@ public class IndexSearcher implements Cl
     }
   }
 
-  /* Sugar for .getIndexReader().document(docID) */
+  /* Sugar for <code>.getIndexReader().document(docID)</code> */
   public Document doc(int docID) throws CorruptIndexException, IOException {
     return reader.document(docID);
   }
-  
-  /* Sugar for .getIndexReader().document(docID, fieldSelector) */
-  public Document doc(int docID, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
-    return reader.document(docID, fieldSelector);
+
+  /* Sugar for <code>.getIndexReader().document(docID, fieldVisitor)</code> */
+  public void doc(int docID, StoredFieldVisitor fieldVisitor) throws CorruptIndexException, IOException {
+    reader.document(docID, fieldVisitor);
   }
-  
+
   /** Expert: Set the SimilarityProvider implementation used by this Searcher.
    *
    */

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java Mon Aug 29 23:13:10 2011
@@ -457,12 +457,8 @@ import org.apache.lucene.util.SmallFloat
  *      <b><i>norm(t,d)</i></b> encapsulates a few (indexing time) boost and length factors:
  *
  *      <ul>
- *        <li><b>Document boost</b> - set by calling
- *        {@link org.apache.lucene.document.Document#setBoost(float) doc.setBoost()}
- *        before adding the document to the index.
- *        </li>
  *        <li><b>Field boost</b> - set by calling
- *        {@link org.apache.lucene.document.Fieldable#setBoost(float) field.setBoost()}
+ *        {@link org.apache.lucene.document.Field#setBoost(float) field.setBoost()}
  *        before adding the field to a document.
  *        </li>
  *        <li><b>lengthNorm</b> - computed
@@ -483,8 +479,6 @@ import org.apache.lucene.util.SmallFloat
  *        <tr>
  *          <td valign="middle" align="right" rowspan="1">
  *            norm(t,d) &nbsp; = &nbsp;
- *            {@link org.apache.lucene.document.Document#getBoost() doc.getBoost()}
- *            &nbsp;&middot;&nbsp;
  *            lengthNorm
  *            &nbsp;&middot;&nbsp;
  *          </td>
@@ -492,7 +486,7 @@ import org.apache.lucene.util.SmallFloat
  *            <big><big><big>&prod;</big></big></big>
  *          </td>
  *          <td valign="middle" align="right" rowspan="1">
- *            {@link org.apache.lucene.document.Fieldable#getBoost() f.getBoost}()
+ *            {@link org.apache.lucene.index.IndexableField#boost() f.boost}()
  *          </td>
  *        </tr>
  *        <tr valigh="top">

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java Mon Aug 29 23:13:10 2011
@@ -68,7 +68,7 @@ public final class CompoundFileDirectory
         success = true;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, handle);
+          IOUtils.closeWhileHandlingException(handle);
         }
       }
       this.isOpen = true;
@@ -95,10 +95,10 @@ public final class CompoundFileDirectory
       if (firstInt == CompoundFileWriter.FORMAT_CURRENT) {
         IndexInput input = null;
         try {
-          input = dir.openInput(IndexFileNames.segmentFileName(
-              IndexFileNames.stripExtension(name), "",
-              IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION),
-              IOContext.READONCE);
+          final String entriesFileName = IndexFileNames.segmentFileName(
+                                                IndexFileNames.stripExtension(name), "",
+                                                IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
+          input = dir.openInput(entriesFileName, IOContext.READONCE);
           final int readInt = input.readInt(); // unused right now
           assert readInt == CompoundFileWriter.ENTRY_FORMAT_CURRENT;
           final int numEntries = input.readVInt();
@@ -112,7 +112,7 @@ public final class CompoundFileDirectory
           }
           return mapping;
         } finally {
-          IOUtils.closeSafely(true, input);
+          IOUtils.close(input);
         }
       } else {
         // TODO remove once 3.x is not supported anymore
@@ -121,7 +121,11 @@ public final class CompoundFileDirectory
       success = true;
       return mapping;
     } finally {
-      IOUtils.closeSafely(!success, stream);
+      if (success) {
+        IOUtils.close(stream);
+      } else {
+        IOUtils.closeWhileHandlingException(stream);
+      }
     }
   }
 
@@ -196,7 +200,7 @@ public final class CompoundFileDirectory
       assert openForWrite;
       writer.close();
     } else {
-      IOUtils.closeSafely(false, handle);
+      IOUtils.close(handle);
     }
   }
   

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java Mon Aug 29 23:13:10 2011
@@ -117,7 +117,7 @@ final class CompoundFileWriter implement
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeSafely(true, dataOut);
+        IOUtils.closeWhileHandlingException(dataOut);
       }
     }
   }
@@ -157,7 +157,7 @@ final class CompoundFileWriter implement
     } catch (IOException e) {
       priorException = e;
     } finally {
-      IOUtils.closeSafely(priorException, dataOut);
+      IOUtils.closeWhileHandlingException(priorException, dataOut);
     }
     try {
       entryTableOut = directory.createOutput(entryTableName, IOContext.DEFAULT);
@@ -165,7 +165,7 @@ final class CompoundFileWriter implement
     } catch (IOException e) {
       priorException = e;
     } finally {
-      IOUtils.closeSafely(priorException, entryTableOut);
+      IOUtils.closeWhileHandlingException(priorException, entryTableOut);
     }
   }
 
@@ -205,13 +205,14 @@ final class CompoundFileWriter implement
       success = true;
       return length;
     } finally {
-      IOUtils.closeSafely(!success, is);
       if (success) {
+        IOUtils.close(is);
         // copy successful - delete file
         fileEntry.dir.deleteFile(fileEntry.file);
+      } else {
+        IOUtils.closeWhileHandlingException(is);
       }
     }
-
   }
 
   protected void writeEntryTable(Collection<FileEntry> entries,

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/Directory.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/Directory.java Mon Aug 29 23:13:10 2011
@@ -204,7 +204,7 @@ public abstract class Directory implemen
     } catch (IOException ioe) {
       priorException = ioe;
     } finally {
-      IOUtils.closeSafely(priorException, os, is);
+      IOUtils.closeWhileHandlingException(priorException, os, is);
     }
   }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/IOUtils.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/IOUtils.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/IOUtils.java Mon Aug 29 23:13:10 2011
@@ -49,7 +49,7 @@ public final class IOUtils {
    * @param priorException  <tt>null</tt> or an exception that will be rethrown after method completion
    * @param objects         objects to call <tt>close()</tt> on
    */
-  public static <E extends Exception> void closeSafely(E priorException, Closeable... objects) throws E, IOException {
+  public static <E extends Exception> void closeWhileHandlingException(E priorException, Closeable... objects) throws E, IOException {
     Throwable th = null;
 
     for (Closeable object : objects) {
@@ -75,8 +75,8 @@ public final class IOUtils {
     }
   }
 
-  /** @see #closeSafely(Exception, Closeable...) */
-  public static <E extends Exception> void closeSafely(E priorException, Iterable<Closeable> objects) throws E, IOException {
+  /** @see #closeWhileHandlingException(Exception, Closeable...) */
+  public static <E extends Exception> void closeWhileHandlingException(E priorException, Iterable<Closeable> objects) throws E, IOException {
     Throwable th = null;
 
     for (Closeable object : objects) {
@@ -103,18 +103,16 @@ public final class IOUtils {
   }
 
   /**
-   * Closes all given <tt>Closeable</tt>s, suppressing all thrown exceptions.
-   * Some of the <tt>Closeable</tt>s may be null, they are ignored. After
-   * everything is closed, and if {@code suppressExceptions} is {@code false},
-   * method either throws the first of suppressed exceptions, or completes
-   * normally.
+   * Closes all given <tt>Closeable</tt>s.  Some of the
+   * <tt>Closeable</tt>s may be null; they are
+   * ignored.  After everything is closed, the method either
+   * throws the first exception it hit while closing, or
+   * completes normally if there were no exceptions.
    * 
-   * @param suppressExceptions
-   *          if true then exceptions that occur during close() are suppressed
    * @param objects
    *          objects to call <tt>close()</tt> on
    */
-  public static void closeSafely(boolean suppressExceptions, Closeable... objects) throws IOException {
+  public static void close(Closeable... objects) throws IOException {
     Throwable th = null;
 
     for (Closeable object : objects) {
@@ -124,12 +122,13 @@ public final class IOUtils {
         }
       } catch (Throwable t) {
         addSuppressed(th, t);
-        if (th == null)
+        if (th == null) {
           th = t;
+        }
       }
     }
 
-    if (th != null && !suppressExceptions) {
+    if (th != null) {
       if (th instanceof IOException) throw (IOException) th;
       if (th instanceof RuntimeException) throw (RuntimeException) th;
       if (th instanceof Error) throw (Error) th;
@@ -138,9 +137,9 @@ public final class IOUtils {
   }
   
   /**
-   * @see #closeSafely(boolean, Closeable...)
+   * @see #close(Closeable...)
    */
-  public static void closeSafely(boolean suppressExceptions, Iterable<? extends Closeable> objects) throws IOException {
+  public static void close(Iterable<? extends Closeable> objects) throws IOException {
     Throwable th = null;
 
     for (Closeable object : objects) {
@@ -150,18 +149,51 @@ public final class IOUtils {
         }
       } catch (Throwable t) {
         addSuppressed(th, t);
-        if (th == null)
+        if (th == null) {
           th = t;
+        }
       }
     }
 
-    if (th != null && !suppressExceptions) {
+    if (th != null) {
       if (th instanceof IOException) throw (IOException) th;
       if (th instanceof RuntimeException) throw (RuntimeException) th;
       if (th instanceof Error) throw (Error) th;
       throw new RuntimeException(th);
     }
   }
+
+  /**
+   * Closes all given <tt>Closeable</tt>s, suppressing all thrown exceptions.
+   * Some of the <tt>Closeable</tt>s may be null, they are ignored.
+   * 
+   * @param objects
+   *          objects to call <tt>close()</tt> on
+   */
+  public static void closeWhileHandlingException(Closeable... objects) throws IOException {
+    for (Closeable object : objects) {
+      try {
+        if (object != null) {
+          object.close();
+        }
+      } catch (Throwable t) {
+      }
+    }
+  }
+  
+  /**
+   * @see #closeWhileHandlingException(Closeable...)
+   */
+  public static void closeWhileHandlingException(Iterable<? extends Closeable> objects) throws IOException {
+    for (Closeable object : objects) {
+      try {
+        if (object != null) {
+          object.close();
+        }
+      } catch (Throwable t) {
+      }
+    }
+  }
   
   /** This reflected {@link Method} is {@code null} before Java 7 */
   private static final Method SUPPRESS_METHOD;

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/CollationTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/CollationTestBase.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/CollationTestBase.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/CollationTestBase.java Mon Aug 29 23:13:10 2011
@@ -18,14 +18,11 @@ package org.apache.lucene.analysis;
  */
 
 
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.ScoreDoc;
@@ -36,8 +33,11 @@ import org.apache.lucene.search.TermRang
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IndexableBinaryStringTools;
 import org.apache.lucene.util.LuceneTestCase;
@@ -81,10 +81,8 @@ public abstract class CollationTestBase 
     IndexWriter writer = new IndexWriter(ramDir, new IndexWriterConfig(
         TEST_VERSION_CURRENT, analyzer));
     Document doc = new Document();
-    doc.add(new Field("content", "\u0633\u0627\u0628", 
-                      Field.Store.YES, Field.Index.ANALYZED));
-    doc.add(new Field("body", "body",
-                      Field.Store.YES, Field.Index.NOT_ANALYZED));
+    doc.add(new Field("content", TextField.TYPE_STORED, "\u0633\u0627\u0628"));
+    doc.add(new Field("body", StringField.TYPE_STORED, "body"));
     writer.addDocument(doc);
     writer.close();
     IndexSearcher searcher = new IndexSearcher(ramDir, true);
@@ -118,8 +116,7 @@ public abstract class CollationTestBase 
     // orders the U+0698 character before the U+0633 character, so the single
     // index Term below should NOT be returned by a TermRangeQuery with a Farsi
     // Collator (or an Arabic one for the case when Farsi is not supported).
-    doc.add(new Field("content", "\u0633\u0627\u0628", 
-                      Field.Store.YES, Field.Index.ANALYZED));
+    doc.add(new Field("content", TextField.TYPE_STORED, "\u0633\u0627\u0628"));
     writer.addDocument(doc);
     writer.close();
     IndexSearcher searcher = new IndexSearcher(ramDir, true);
@@ -141,10 +138,8 @@ public abstract class CollationTestBase 
     IndexWriter writer = new IndexWriter(farsiIndex, new IndexWriterConfig(
         TEST_VERSION_CURRENT, analyzer));
     Document doc = new Document();
-    doc.add(new Field("content", "\u0633\u0627\u0628", 
-                      Field.Store.YES, Field.Index.ANALYZED));
-    doc.add(new Field("body", "body",
-                      Field.Store.YES, Field.Index.NOT_ANALYZED));
+    doc.add(new Field("content", TextField.TYPE_STORED, "\u0633\u0627\u0628"));
+    doc.add(new Field("body", StringField.TYPE_STORED, "body"));
     writer.addDocument(doc);
     writer.close();
 
@@ -204,20 +199,21 @@ public abstract class CollationTestBase 
       {  "J",   "y",     "HOT",             "HOT",             "HOT",             "HOT"             },
     };
 
+    FieldType customType = new FieldType();
+    customType.setStored(true);
+    
     for (int i = 0 ; i < sortData.length ; ++i) {
       Document doc = new Document();
-      doc.add(new Field("tracer", sortData[i][0], 
-                        Field.Store.YES, Field.Index.NO));
-      doc.add(new Field("contents", sortData[i][1], 
-                        Field.Store.NO, Field.Index.ANALYZED));
+      doc.add(new Field("tracer", customType, sortData[i][0]));
+      doc.add(new TextField("contents", sortData[i][1]));
       if (sortData[i][2] != null) 
-        doc.add(new Field("US", usAnalyzer.reusableTokenStream("US", new StringReader(sortData[i][2]))));
+        doc.add(new TextField("US", usAnalyzer.reusableTokenStream("US", new StringReader(sortData[i][2]))));
       if (sortData[i][3] != null) 
-        doc.add(new Field("France", franceAnalyzer.reusableTokenStream("France", new StringReader(sortData[i][3]))));
+        doc.add(new TextField("France", franceAnalyzer.reusableTokenStream("France", new StringReader(sortData[i][3]))));
       if (sortData[i][4] != null)
-        doc.add(new Field("Sweden", swedenAnalyzer.reusableTokenStream("Sweden", new StringReader(sortData[i][4]))));
+        doc.add(new TextField("Sweden", swedenAnalyzer.reusableTokenStream("Sweden", new StringReader(sortData[i][4]))));
       if (sortData[i][5] != null) 
-        doc.add(new Field("Denmark", denmarkAnalyzer.reusableTokenStream("Denmark", new StringReader(sortData[i][5]))));
+        doc.add(new TextField("Denmark", denmarkAnalyzer.reusableTokenStream("Denmark", new StringReader(sortData[i][5]))));
       writer.addDocument(doc);
     }
     writer.optimize();
@@ -250,9 +246,9 @@ public abstract class CollationTestBase 
     int n = result.length;
     for (int i = 0 ; i < n ; ++i) {
       Document doc = searcher.doc(result[i].doc);
-      String[] v = doc.getValues("tracer");
+      IndexableField[] v = doc.getFields("tracer");
       for (int j = 0 ; j < v.length ; ++j) {
-        buff.append(v[j]);
+        buff.append(v[j].stringValue());
       }
     }
     assertEquals(expectedResult, buff.toString());

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/DocHelper.java Mon Aug 29 23:13:10 2011
@@ -26,77 +26,114 @@ import java.util.Random;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.BinaryField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.Fieldable;
-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.StringField;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
 
 import static org.apache.lucene.util.LuceneTestCase.TEST_VERSION_CURRENT;
 
 class DocHelper {
+  
+  public static final FieldType customType;
   public static final String FIELD_1_TEXT = "field one text";
   public static final String TEXT_FIELD_1_KEY = "textField1";
-  public static Field textField1 = new Field(TEXT_FIELD_1_KEY, FIELD_1_TEXT,
-      Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.NO);
-  
+  public static Field textField1;
+  static {
+    customType = new FieldType(TextField.TYPE_STORED);
+    textField1 = new Field(TEXT_FIELD_1_KEY, customType, FIELD_1_TEXT);
+  }
+
+  public static final FieldType customType2;
   public static final String FIELD_2_TEXT = "field field field two text";
   //Fields will be lexicographically sorted.  So, the order is: field, text, two
   public static final int [] FIELD_2_FREQS = {3, 1, 1}; 
   public static final String TEXT_FIELD_2_KEY = "textField2";
-  public static Field textField2 = new Field(TEXT_FIELD_2_KEY, FIELD_2_TEXT, Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
+  public static Field textField2;
+  static {
+    customType2 = new FieldType(TextField.TYPE_STORED);
+    customType2.setStoreTermVectors(true);
+    customType2.setStoreTermVectorPositions(true);
+    customType2.setStoreTermVectorOffsets(true);
+    textField2 = new Field(TEXT_FIELD_2_KEY, customType2, FIELD_2_TEXT);
+  }
   
+  public static final FieldType customType3;
   public static final String FIELD_3_TEXT = "aaaNoNorms aaaNoNorms bbbNoNorms";
   public static final String TEXT_FIELD_3_KEY = "textField3";
-  public static Field textField3 = new Field(TEXT_FIELD_3_KEY, FIELD_3_TEXT, Field.Store.YES, Field.Index.ANALYZED);
-  static { textField3.setOmitNorms(true); }
+  public static Field textField3;
+  
+  static {
+    customType3 = new FieldType(TextField.TYPE_STORED);
+    customType3.setOmitNorms(true);
+    textField3 = new Field(TEXT_FIELD_3_KEY, customType3, FIELD_3_TEXT);
+  }
 
   public static final String KEYWORD_TEXT = "Keyword";
   public static final String KEYWORD_FIELD_KEY = "keyField";
-  public static Field keyField = new Field(KEYWORD_FIELD_KEY, KEYWORD_TEXT,
-      Field.Store.YES, Field.Index.NOT_ANALYZED);
+  public static Field keyField;
+  static {
+    keyField = new Field(KEYWORD_FIELD_KEY, StringField.TYPE_STORED,  KEYWORD_TEXT);
+  }
 
+  public static final FieldType customType5;
   public static final String NO_NORMS_TEXT = "omitNormsText";
   public static final String NO_NORMS_KEY = "omitNorms";
-  public static Field noNormsField = new Field(NO_NORMS_KEY, NO_NORMS_TEXT,
-      Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
+  public static Field noNormsField;
+  static {
+    customType5 = new FieldType(TextField.TYPE_STORED);
+    customType5.setOmitNorms(true);
+    customType5.setTokenized(false);
+    noNormsField = new Field(NO_NORMS_KEY, customType5, NO_NORMS_TEXT);
+  }
 
+  public static final FieldType customType6;
   public static final String NO_TF_TEXT = "analyzed with no tf and positions";
   public static final String NO_TF_KEY = "omitTermFreqAndPositions";
-  public static Field noTFField = new Field(NO_TF_KEY, NO_TF_TEXT,
-      Field.Store.YES, Field.Index.ANALYZED);
+  public static Field noTFField;
   static {
-    noTFField.setIndexOptions(IndexOptions.DOCS_ONLY);
+    customType6 = new FieldType(TextField.TYPE_STORED);
+    customType6.setIndexOptions(IndexOptions.DOCS_ONLY);
+    noTFField = new Field(NO_TF_KEY, customType6, NO_TF_TEXT);
   }
 
+  public static final FieldType customType7;
   public static final String UNINDEXED_FIELD_TEXT = "unindexed field text";
   public static final String UNINDEXED_FIELD_KEY = "unIndField";
-  public static Field unIndField = new Field(UNINDEXED_FIELD_KEY, UNINDEXED_FIELD_TEXT,
-      Field.Store.YES, Field.Index.NO);
+  public static Field unIndField;
+  static {
+    customType7 = new FieldType();
+    customType7.setStored(true);
+    unIndField = new Field(UNINDEXED_FIELD_KEY, customType7, UNINDEXED_FIELD_TEXT);
+  }
 
 
   public static final String UNSTORED_1_FIELD_TEXT = "unstored field text";
   public static final String UNSTORED_FIELD_1_KEY = "unStoredField1";
-  public static Field unStoredField1 = new Field(UNSTORED_FIELD_1_KEY, UNSTORED_1_FIELD_TEXT,
-      Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.NO);
+  public static Field unStoredField1 = new Field(UNSTORED_FIELD_1_KEY, TextField.TYPE_UNSTORED, UNSTORED_1_FIELD_TEXT);
 
+  public static final FieldType customType8;
   public static final String UNSTORED_2_FIELD_TEXT = "unstored field text";
   public static final String UNSTORED_FIELD_2_KEY = "unStoredField2";
-  public static Field unStoredField2 = new Field(UNSTORED_FIELD_2_KEY, UNSTORED_2_FIELD_TEXT,
-      Field.Store.NO, Field.Index.ANALYZED, Field.TermVector.YES);
+  public static Field unStoredField2;
+  static {
+    customType8 = new FieldType(TextField.TYPE_UNSTORED);
+    customType8.setStoreTermVectors(true);
+    unStoredField2 = new Field(UNSTORED_FIELD_2_KEY, customType8, UNSTORED_2_FIELD_TEXT);
+  }
 
   public static final String LAZY_FIELD_BINARY_KEY = "lazyFieldBinary";
   public static byte [] LAZY_FIELD_BINARY_BYTES;
   public static Field lazyFieldBinary;
-  
+
   public static final String LAZY_FIELD_KEY = "lazyField";
   public static final String LAZY_FIELD_TEXT = "These are some field bytes";
-  public static Field lazyField = new Field(LAZY_FIELD_KEY, LAZY_FIELD_TEXT, Field.Store.YES, Field.Index.ANALYZED);
+  public static Field lazyField = new Field(LAZY_FIELD_KEY, customType, LAZY_FIELD_TEXT);
   
   public static final String LARGE_LAZY_FIELD_KEY = "largeLazyField";
   public static String LARGE_LAZY_FIELD_TEXT;
@@ -105,15 +142,13 @@ class DocHelper {
   //From Issue 509
   public static final String FIELD_UTF1_TEXT = "field one \u4e00text";
   public static final String TEXT_FIELD_UTF1_KEY = "textField1Utf8";
-  public static Field textUtfField1 = new Field(TEXT_FIELD_UTF1_KEY, FIELD_UTF1_TEXT,
-      Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.NO);
+  public static Field textUtfField1 = new Field(TEXT_FIELD_UTF1_KEY, customType, FIELD_UTF1_TEXT);
 
   public static final String FIELD_UTF2_TEXT = "field field field \u4e00two text";
   //Fields will be lexicographically sorted.  So, the order is: field, text, two
   public static final int [] FIELD_UTF2_FREQS = {3, 1, 1};
   public static final String TEXT_FIELD_UTF2_KEY = "textField2Utf8";
-  public static Field textUtfField2 = new Field(TEXT_FIELD_UTF2_KEY, FIELD_UTF2_TEXT, Field.Store.YES, 
-          Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
+  public static Field textUtfField2 = new Field(TEXT_FIELD_UTF2_KEY, customType2, FIELD_UTF2_TEXT);
  
   
   
@@ -139,16 +174,16 @@ class DocHelper {
     largeLazyField//placeholder for large field, since this is null.  It must always be last
   };
 
-  public static Map<String,Fieldable> all     =new HashMap<String,Fieldable>();
-  public static Map<String,Fieldable> indexed =new HashMap<String,Fieldable>();
-  public static Map<String,Fieldable> stored  =new HashMap<String,Fieldable>();
-  public static Map<String,Fieldable> unstored=new HashMap<String,Fieldable>();
-  public static Map<String,Fieldable> unindexed=new HashMap<String,Fieldable>();
-  public static Map<String,Fieldable> termvector=new HashMap<String,Fieldable>();
-  public static Map<String,Fieldable> notermvector=new HashMap<String,Fieldable>();
-  public static Map<String,Fieldable> lazy= new HashMap<String,Fieldable>();
-  public static Map<String,Fieldable> noNorms=new HashMap<String,Fieldable>();
-  public static Map<String,Fieldable> noTf=new HashMap<String,Fieldable>();
+  public static Map<String,IndexableField> all     =new HashMap<String,IndexableField>();
+  public static Map<String,IndexableField> indexed =new HashMap<String,IndexableField>();
+  public static Map<String,IndexableField> stored  =new HashMap<String,IndexableField>();
+  public static Map<String,IndexableField> unstored=new HashMap<String,IndexableField>();
+  public static Map<String,IndexableField> unindexed=new HashMap<String,IndexableField>();
+  public static Map<String,IndexableField> termvector=new HashMap<String,IndexableField>();
+  public static Map<String,IndexableField> notermvector=new HashMap<String,IndexableField>();
+  public static Map<String,IndexableField> lazy= new HashMap<String,IndexableField>();
+  public static Map<String,IndexableField> noNorms=new HashMap<String,IndexableField>();
+  public static Map<String,IndexableField> noTf=new HashMap<String,IndexableField>();
 
   static {
     //Initialize the large Lazy Field
@@ -162,28 +197,29 @@ class DocHelper {
       LAZY_FIELD_BINARY_BYTES = "These are some binary field bytes".getBytes("UTF8");
     } catch (UnsupportedEncodingException e) {
     }
-    lazyFieldBinary = new Field(LAZY_FIELD_BINARY_KEY, LAZY_FIELD_BINARY_BYTES);
+    lazyFieldBinary = new BinaryField(LAZY_FIELD_BINARY_KEY, LAZY_FIELD_BINARY_BYTES);
     fields[fields.length - 2] = lazyFieldBinary;
     LARGE_LAZY_FIELD_TEXT = buffer.toString();
-    largeLazyField = new Field(LARGE_LAZY_FIELD_KEY, LARGE_LAZY_FIELD_TEXT, Field.Store.YES, Field.Index.ANALYZED);
+    largeLazyField = new Field(LARGE_LAZY_FIELD_KEY, customType, LARGE_LAZY_FIELD_TEXT);
     fields[fields.length - 1] = largeLazyField;
     for (int i=0; i<fields.length; i++) {
-      Fieldable f = fields[i];
+      IndexableField f = fields[i];
       add(all,f);
-      if (f.isIndexed()) add(indexed,f);
+      if (f.indexed()) add(indexed,f);
       else add(unindexed,f);
-      if (f.isTermVectorStored()) add(termvector,f);
-      if (f.isIndexed() && !f.isTermVectorStored()) add(notermvector,f);
-      if (f.isStored()) add(stored,f);
+      if (f.storeTermVectors()) add(termvector,f);
+      if (f.indexed() && !f.storeTermVectors()) add(notermvector,f);
+      if (f.stored()) add(stored,f);
       else add(unstored,f);
-      if (f.getOmitNorms()) add(noNorms,f);
-      if (f.getIndexOptions() == IndexOptions.DOCS_ONLY) add(noTf,f);
-      if (f.isLazy()) add(lazy, f);
+      if (f.indexOptions() == IndexOptions.DOCS_ONLY) add(noTf,f);
+      if (f.omitNorms()) add(noNorms,f);
+      if (f.indexOptions() == IndexOptions.DOCS_ONLY) add(noTf,f);
+      //if (f.isLazy()) add(lazy, f);
     }
   }
 
 
-  private static void add(Map<String,Fieldable> map, Fieldable field) {
+  private static void add(Map<String,IndexableField> map, IndexableField field) {
     map.put(field.name(), field);
   }
 
@@ -257,17 +293,26 @@ class DocHelper {
   
   public static Document createDocument(int n, String indexName, int numFields) {
     StringBuilder sb = new StringBuilder();
-    Document doc = new Document();
-    doc.add(new Field("id", Integer.toString(n), Store.YES, Index.NOT_ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
-    doc.add(new Field("indexname", indexName, Store.YES, Index.NOT_ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
+    FieldType customType = new FieldType(TextField.TYPE_STORED);
+    customType.setStoreTermVectors(true);
+    customType.setStoreTermVectorPositions(true);
+    customType.setStoreTermVectorOffsets(true);
+
+    FieldType customType1 = new FieldType(StringField.TYPE_STORED);
+    customType1.setStoreTermVectors(true);
+    customType1.setStoreTermVectorPositions(true);
+    customType1.setStoreTermVectorOffsets(true);
+
+    final Document doc = new Document();
+    doc.add(new Field("id", customType1, Integer.toString(n)));
+    doc.add(new Field("indexname", customType1, indexName));
     sb.append("a");
     sb.append(n);
-    doc.add(new Field("field1", sb.toString(), Store.YES, Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
+    doc.add(new Field("field1", customType, sb.toString()));
     sb.append(" b");
     sb.append(n);
     for (int i = 1; i < numFields; i++) {
-      doc.add(new Field("field" + (i + 1), sb.toString(), Store.YES,
-                        Index.ANALYZED, TermVector.WITH_POSITIONS_OFFSETS));
+      doc.add(new Field("field" + (i + 1), customType, sb.toString()));
     }
     return doc;
   }

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/RandomCodecProvider.java Mon Aug 29 23:13:10 2011
@@ -54,15 +54,15 @@ public class RandomCodecProvider extends
     // block via CL:
     int minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
     int maxItemsPerBlock = 2*(Math.max(2, minItemsPerBlock-1)) + random.nextInt(100);
-    register(_TestUtil.randomizeCodec(random, new StandardCodec(minItemsPerBlock, maxItemsPerBlock)));
-    register(_TestUtil.randomizeCodec(random, new PreFlexCodec()));
+    register(new StandardCodec(minItemsPerBlock, maxItemsPerBlock));
+    register(new PreFlexCodec());
     // TODO: make it possible to specify min/max iterms per
     // block via CL:
     minItemsPerBlock = _TestUtil.nextInt(random, 2, 100);
     maxItemsPerBlock = 2*(Math.max(1, minItemsPerBlock-1)) + random.nextInt(100);
-    register(_TestUtil.randomizeCodec(random, new PulsingCodec( 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock)));
-    register(_TestUtil.randomizeCodec(random, new SimpleTextCodec()));
-    register(_TestUtil.randomizeCodec(random, new MemoryCodec()));
+    register(new PulsingCodec( 1 + random.nextInt(20), minItemsPerBlock, maxItemsPerBlock));
+    register(new SimpleTextCodec());
+    register(new MemoryCodec());
     Collections.shuffle(knownCodecs, random);
   }
   

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Mon Aug 29 23:13:10 2011
@@ -120,23 +120,22 @@ public class RandomIndexWriter implement
   
   /**
    * Adds a Document.
-   * @see IndexWriter#addDocument(Document)
+   * @see IndexWriter#addDocument(Iterable)
    */
-  public void addDocument(final Document doc) throws IOException {
-    if (doDocValues) {
-      randomPerDocFieldValues(r, doc);
+  public <T extends IndexableField> void addDocument(final Iterable<T> doc) throws IOException {
+    if (doDocValues && doc instanceof Document) {
+      randomPerDocFieldValues(r, (Document) doc);
     }
-
     if (r.nextInt(5) == 3) {
       // TODO: maybe, we should simply buffer up added docs
       // (but we need to clone them), and only when
       // getReader, commit, etc. are called, we do an
       // addDocuments?  Would be better testing.
-      w.addDocuments(new Iterable<Document>() {
+      w.addDocuments(new Iterable<Iterable<T>>() {
 
         @Override
-        public Iterator<Document> iterator() {
-          return new Iterator<Document>() {
+        public Iterator<Iterable<T>> iterator() {
+          return new Iterator<Iterable<T>>() {
             boolean done;
             
             @Override
@@ -150,7 +149,7 @@ public class RandomIndexWriter implement
             }
 
             @Override
-            public Document next() {
+            public Iterable<T> next() {
               if (done) {
                 throw new IllegalStateException();
               }
@@ -172,7 +171,7 @@ public class RandomIndexWriter implement
     ValueType[] values = ValueType.values();
     ValueType type = values[random.nextInt(values.length)];
     String name = "random_" + type.name() + "" + docValuesFieldPrefix;
-    if ("PreFlex".equals(codecProvider.getFieldCodec(name)) || doc.getFieldable(name) != null)
+    if ("PreFlex".equals(codecProvider.getFieldCodec(name)) || doc.getField(name) != null)
         return;
     IndexDocValuesField docValuesField = new IndexDocValuesField(name);
     switch (type) {
@@ -238,31 +237,30 @@ public class RandomIndexWriter implement
     }
   }
   
-  public void addDocuments(Iterable<Document> docs) throws IOException {
+  public void addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
     w.addDocuments(docs);
     maybeCommit();
   }
 
-  public void updateDocuments(Term delTerm, Iterable<Document> docs) throws IOException {
+  public void updateDocuments(Term delTerm, Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
     w.updateDocuments(delTerm, docs);
     maybeCommit();
   }
 
   /**
    * Updates a document.
-   * @see IndexWriter#updateDocument(Term, Document)
+   * @see IndexWriter#updateDocument(Term, Iterable)
    */
-  public void updateDocument(final Term t, final Document doc) throws IOException {
+  public <T extends IndexableField> void updateDocument(Term t, final Iterable<T> doc) throws IOException {
     if (doDocValues) {
-      randomPerDocFieldValues(r, doc);
+      randomPerDocFieldValues(r, (Document) doc);
     }
-    
     if (r.nextInt(5) == 3) {
-      w.updateDocuments(t, new Iterable<Document>() {
+      w.updateDocuments(t, new Iterable<Iterable<T>>() {
 
         @Override
-        public Iterator<Document> iterator() {
-          return new Iterator<Document>() {
+        public Iterator<Iterable<T>> iterator() {
+          return new Iterator<Iterable<T>>() {
             boolean done;
             
             @Override
@@ -276,7 +274,7 @@ public class RandomIndexWriter implement
             }
 
             @Override
-            public Document next() {
+            public Iterable<T> next() {
               if (done) {
                 throw new IllegalStateException();
               }

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java Mon Aug 29 23:13:10 2011
@@ -119,7 +119,7 @@ public class MockFixedIntBlockCodec exte
         return ret;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, out);
+          IOUtils.closeWhileHandlingException(out);
         }
       }
     }

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java Mon Aug 29 23:13:10 2011
@@ -142,7 +142,7 @@ public class MockVariableIntBlockCodec e
         return ret;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, out);
+          IOUtils.closeWhileHandlingException(out);
         }
       }
     }

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Mon Aug 29 23:13:10 2011
@@ -80,7 +80,12 @@ public class MockRandomCodec extends Cod
   private final String SEED_EXT = "sd";
   
   public MockRandomCodec(Random random) {
-    super("MockRandom");
+    this(random, "MockRandom", true);
+    
+  }
+  
+  protected MockRandomCodec(Random random, String name, boolean docValuesUseCompoundFile) {
+    super(name, docValuesUseCompoundFile);
     this.seedRandom = new Random(random.nextLong());
   }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java Mon Aug 29 23:13:10 2011
@@ -44,7 +44,7 @@ public class MockSingleIntIndexOutput ex
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeSafely(true, out);
+        IOUtils.closeWhileHandlingException(out);
       }
     }
   }

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java Mon Aug 29 23:13:10 2011
@@ -78,7 +78,7 @@ class PreFlexFieldsWriter extends Fields
 
   @Override
   public void close() throws IOException {
-    IOUtils.closeSafely(false, termsOut, freqOut, proxOut);
+    IOUtils.close(termsOut, freqOut, proxOut);
   }
 
   private class PreFlexTermsWriter extends TermsConsumer {

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/preflexrw/TermInfosWriter.java Mon Aug 29 23:13:10 2011
@@ -98,7 +98,7 @@ final class TermInfosWriter implements C
     } finally {
       if (!success) {
         try {
-          IOUtils.closeSafely(true, output);
+          IOUtils.closeWhileHandlingException(output);
         } catch (IOException e) {
           // cannot happen since we suppress exceptions
           throw new RuntimeException(e);
@@ -139,7 +139,7 @@ final class TermInfosWriter implements C
     } finally {
       if (!success) {
         try {
-          IOUtils.closeSafely(true, output);
+          IOUtils.closeWhileHandlingException(output);
         } catch (IOException e) {
           // cannot happen since we suppress exceptions
           throw new RuntimeException(e);

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java Mon Aug 29 23:13:10 2011
@@ -153,7 +153,10 @@ public class MockDirectoryWrapper extend
   
   @Override
   public String toString() {
-    maybeYield();
+    // NOTE: do not maybeYield here, since it consumes
+    // randomness and can thus (unexpectedly during
+    // debugging) change the behavior of a seed
+    // maybeYield();
     return "MockDirWrapper(" + delegate + ")";
   }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/util/LineFileDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/util/LineFileDocs.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/util/LineFileDocs.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/util/LineFileDocs.java Mon Aug 29 23:13:10 2011
@@ -30,6 +30,9 @@ import java.util.Random;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
 
 /** Minimal port of contrib/benchmark's LneDocSource +
  * DocMaker, so tests can enum docs from a line file created
@@ -117,19 +120,24 @@ public class LineFileDocs implements Clo
     public DocState() {
       doc = new Document();
       
-      title = new Field("title", "", Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS);
+      title = new StringField("title", "");
       doc.add(title);
 
-      titleTokenized = new Field("titleTokenized", "", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
+      FieldType ft = new FieldType(TextField.TYPE_STORED);
+      ft.setStoreTermVectors(true);
+      ft.setStoreTermVectorOffsets(true);
+      ft.setStoreTermVectorPositions(true);
+      
+      titleTokenized = new Field("titleTokenized", ft, "");
       doc.add(titleTokenized);
 
-      body = new Field("body", "", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
+      body = new Field("body", ft, "");
       doc.add(body);
 
-      id = new Field("docid", "", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
+      id = new Field("docid", StringField.TYPE_STORED, "");
       doc.add(id);
 
-      date = new Field("date", "", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS);
+      date = new Field("date", StringField.TYPE_STORED, "");
       doc.add(date);
     }
   }