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 [4/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/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Mon Aug 29 23:13:10 2011
@@ -27,7 +27,6 @@ import java.util.Queue;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
@@ -320,7 +319,7 @@ final class DocumentsWriter {
     return maybeMerge;
   }
 
-  boolean updateDocuments(final Iterable<Document> docs, final Analyzer analyzer,
+  boolean updateDocuments(final Iterable<? extends Iterable<? extends IndexableField>> docs, final Analyzer analyzer,
                           final Term delTerm) throws CorruptIndexException, IOException {
     boolean maybeMerge = preUpdate();
 
@@ -351,7 +350,7 @@ final class DocumentsWriter {
     return postUpdate(flushingDWPT, maybeMerge);
   }
 
-  boolean updateDocument(final Document doc, final Analyzer analyzer,
+  boolean updateDocument(final Iterable<? extends IndexableField> doc, final Analyzer analyzer,
       final Term delTerm) throws CorruptIndexException, IOException {
 
     boolean maybeMerge = preUpdate();

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Mon Aug 29 23:13:10 2011
@@ -26,7 +26,6 @@ import java.text.NumberFormat;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice;
 import org.apache.lucene.search.similarities.SimilarityProvider;
 import org.apache.lucene.store.Directory;
@@ -90,7 +89,7 @@ public class DocumentsWriterPerThread {
     PrintStream infoStream;
     SimilarityProvider similarityProvider;
     int docID;
-    Document doc;
+    Iterable<? extends IndexableField> doc;
     String maxTermPrefix;
 
     DocState(DocumentsWriterPerThread docWriter) {
@@ -213,7 +212,7 @@ public class DocumentsWriterPerThread {
     return retval;
   }
 
-  public void updateDocument(Document doc, Analyzer analyzer, Term delTerm) throws IOException {
+  public void updateDocument(Iterable<? extends IndexableField> doc, Analyzer analyzer, Term delTerm) throws IOException {
     assert writer.testPoint("DocumentsWriterPerThread addDocument start");
     assert deleteQueue != null;
     docState.doc = doc;
@@ -263,7 +262,7 @@ public class DocumentsWriterPerThread {
     finishDocument(delTerm);
   }
   
-  public int updateDocuments(Iterable<Document> docs, Analyzer analyzer, Term delTerm) throws IOException {
+  public int updateDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer analyzer, Term delTerm) throws IOException {
     assert writer.testPoint("DocumentsWriterPerThread addDocuments start");
     assert deleteQueue != null;
     docState.analyzer = analyzer;
@@ -280,7 +279,7 @@ public class DocumentsWriterPerThread {
     }
     int docCount = 0;
     try {
-      for(Document doc : docs) {
+      for(Iterable<? extends IndexableField> doc : docs) {
         docState.doc = doc;
         docState.docID = numDocsInRAM;
         docCount++;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInfo.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInfo.java Mon Aug 29 23:13:10 2011
@@ -30,9 +30,9 @@ public final class FieldInfo {
 
 
   // true if term vector for this field should be stored
-  boolean storeTermVector;
-  boolean storeOffsetWithTermVector;
-  boolean storePositionWithTermVector;
+  public boolean storeTermVector;
+  public boolean storeOffsetWithTermVector;
+  public boolean storePositionWithTermVector;
 
   public boolean omitNorms; // omit norms associated with indexed fields  
   public IndexOptions indexOptions;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInfos.java Mon Aug 29 23:13:10 2011
@@ -39,8 +39,8 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.CodecUtil;
 
-/** Access to the Fieldable Info file that describes document fields and whether or
- *  not they are indexed. Each segment has a separate Fieldable Info file. Objects
+/** Access to the Field Info file that describes document fields and whether or
+ *  not they are indexed. Each segment has a separate Field Info file. Objects
  *  of this class are thread-safe for multiple readers, but only one thread can
  *  be adding documents at a time, with no other reader or writer threads
  *  accessing this object.
@@ -381,7 +381,7 @@ public final class FieldInfos implements
   /**
    * Calls 5 parameter add with false for all TermVector parameters.
    * 
-   * @param name The name of the Fieldable
+   * @param name The name of the IndexableField
    * @param isIndexed true if the field is indexed
    * @see #addOrUpdate(String, boolean, boolean, boolean, boolean)
    */

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInvertState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInvertState.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInvertState.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldInvertState.java Mon Aug 29 23:13:10 2011
@@ -50,14 +50,14 @@ public final class FieldInvertState {
    * Re-initialize the state, using this boost value.
    * @param docBoost boost value to use.
    */
-  void reset(float docBoost) {
+  void reset() {
     position = 0;
     length = 0;
     numOverlap = 0;
     offset = 0;
     maxTermFrequency = 0;
     uniqueTermCount = 0;
-    boost = docBoost;
+    boost = 1.0f;
     attributeSource = null;
   }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsReader.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsReader.java Mon Aug 29 23:13:10 2011
@@ -17,16 +17,9 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.document.AbstractField;
-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.Fieldable;
-import org.apache.lucene.document.NumericField;
+import java.io.IOException;
+
 import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -34,9 +27,6 @@ import org.apache.lucene.util.CloseableT
 import org.apache.lucene.util.IOUtils;
 
 import java.io.Closeable;
-import java.io.IOException;
-import java.io.Reader;
-import java.util.ArrayList;
 
 /**
  * Class responsible for access to stored document fields.
@@ -49,7 +39,8 @@ public final class FieldsReader implemen
   private final static int FORMAT_SIZE = 4;
 
   private final FieldInfos fieldInfos;
-
+  private CloseableThreadLocal<IndexInput> fieldsStreamTL = new CloseableThreadLocal<IndexInput>();
+  
   // The main fieldStream, used only for cloning.
   private final IndexInput cloneableFieldsStream;
 
@@ -68,7 +59,6 @@ public final class FieldsReader implemen
   // file.  This will be 0 if we have our own private file.
   private int docStoreOffset;
 
-  private CloseableThreadLocal<IndexInput> fieldsStreamTL = new CloseableThreadLocal<IndexInput>();
   private boolean isOriginal = false;
 
   /** Returns a cloned FieldsReader that shares open
@@ -184,9 +174,9 @@ public final class FieldsReader implemen
   public final void close() throws IOException {
     if (!closed) {
       if (isOriginal) {
-        IOUtils.closeSafely(false, fieldsStream, indexStream, fieldsStreamTL, cloneableFieldsStream, cloneableIndexStream);
+        IOUtils.close(fieldsStream, indexStream, fieldsStreamTL, cloneableFieldsStream, cloneableIndexStream);
       } else {
-        IOUtils.closeSafely(false, fieldsStream, indexStream, fieldsStreamTL);
+        IOUtils.close(fieldsStream, indexStream, fieldsStreamTL);
       }
       closed = true;
     }
@@ -200,50 +190,52 @@ public final class FieldsReader implemen
     indexStream.seek(FORMAT_SIZE + (docID + docStoreOffset) * 8L);
   }
 
-  public final Document doc(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
+  public final void visitDocument(int n, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
     seekIndex(n);
-    long position = indexStream.readLong();
-    fieldsStream.seek(position);
+    fieldsStream.seek(indexStream.readLong());
 
-    Document doc = new Document();
-    int numFields = fieldsStream.readVInt();
-    out: for (int i = 0; i < numFields; i++) {
+    final int numFields = fieldsStream.readVInt();
+    for (int fieldIDX = 0; fieldIDX < numFields; fieldIDX++) {
       int fieldNumber = fieldsStream.readVInt();
-      FieldInfo fi = fieldInfos.fieldInfo(fieldNumber);
-      FieldSelectorResult acceptField = fieldSelector == null ? FieldSelectorResult.LOAD : fieldSelector.accept(fi.name);
+      FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
       
       int bits = fieldsStream.readByte() & 0xFF;
-      assert bits <= (FieldsWriter.FIELD_IS_NUMERIC_MASK | FieldsWriter.FIELD_IS_TOKENIZED | FieldsWriter.FIELD_IS_BINARY): "bits=" + Integer.toHexString(bits);
+      assert bits <= (FieldsWriter.FIELD_IS_NUMERIC_MASK | FieldsWriter.FIELD_IS_BINARY): "bits=" + Integer.toHexString(bits);
 
-      boolean tokenize = (bits & FieldsWriter.FIELD_IS_TOKENIZED) != 0;
-      boolean binary = (bits & FieldsWriter.FIELD_IS_BINARY) != 0;
+      final boolean binary = (bits & FieldsWriter.FIELD_IS_BINARY) != 0;
       final int numeric = bits & FieldsWriter.FIELD_IS_NUMERIC_MASK;
 
-      switch (acceptField) {
-        case LOAD:
-          addField(doc, fi, binary, tokenize, numeric);
+      final boolean doStop;
+      if (binary) {
+        final int numBytes = fieldsStream.readVInt();
+        doStop = visitor.binaryField(fieldInfo, fieldsStream, numBytes);
+      } else if (numeric != 0) {
+        switch(numeric) {
+        case FieldsWriter.FIELD_IS_NUMERIC_INT:
+          doStop = visitor.intField(fieldInfo, fieldsStream.readInt());
           break;
-        case LOAD_AND_BREAK:
-          addField(doc, fi, binary, tokenize, numeric);
-          break out; //Get out of this loop
-        case LAZY_LOAD:
-          addFieldLazy(doc, fi, binary, tokenize, true, numeric);
+        case FieldsWriter.FIELD_IS_NUMERIC_LONG:
+          doStop = visitor.longField(fieldInfo, fieldsStream.readLong());
           break;
-        case LATENT:
-          addFieldLazy(doc, fi, binary, tokenize, false, numeric);
+        case FieldsWriter.FIELD_IS_NUMERIC_FLOAT:
+          doStop = visitor.floatField(fieldInfo, Float.intBitsToFloat(fieldsStream.readInt()));
           break;
-        case SIZE:
-          skipFieldBytes(addFieldSize(doc, fi, binary, numeric));
+        case FieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
+          doStop = visitor.doubleField(fieldInfo, Double.longBitsToDouble(fieldsStream.readLong()));
           break;
-        case SIZE_AND_BREAK:
-          addFieldSize(doc, fi, binary, numeric);
-          break out; //Get out of this loop
         default:
-          skipField(numeric);
+          throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
+        }
+      } else {
+        // Text:
+        final int numUTF8Bytes = fieldsStream.readVInt();
+        doStop = visitor.stringField(fieldInfo, fieldsStream, numUTF8Bytes);
       }
-    }
 
-    return doc;
+      if (doStop) {
+        return;
+      }
+    }
   }
 
   /** Returns the length in bytes of each raw document in a
@@ -300,225 +292,4 @@ public final class FieldsReader implemen
   private void skipFieldBytes(int toRead) throws IOException {
     fieldsStream.seek(fieldsStream.getFilePointer() + toRead);
   }
-
-  private NumericField loadNumericField(FieldInfo fi, int numeric) throws IOException {
-    assert numeric != 0;
-    switch(numeric) {
-      case FieldsWriter.FIELD_IS_NUMERIC_INT:
-        return new NumericField(fi.name, Field.Store.YES, fi.isIndexed).setIntValue(fieldsStream.readInt());
-      case FieldsWriter.FIELD_IS_NUMERIC_LONG:
-        return new NumericField(fi.name, Field.Store.YES, fi.isIndexed).setLongValue(fieldsStream.readLong());
-      case FieldsWriter.FIELD_IS_NUMERIC_FLOAT:
-        return new NumericField(fi.name, Field.Store.YES, fi.isIndexed).setFloatValue(Float.intBitsToFloat(fieldsStream.readInt()));
-      case FieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
-        return new NumericField(fi.name, Field.Store.YES, fi.isIndexed).setDoubleValue(Double.longBitsToDouble(fieldsStream.readLong()));
-      default:
-        throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
-    }
-  }
-
-  private void addFieldLazy(Document doc, FieldInfo fi, boolean binary, boolean tokenize, boolean cacheResult, int numeric) throws IOException {
-    final AbstractField f;
-    if (binary) {
-      int toRead = fieldsStream.readVInt();
-      long pointer = fieldsStream.getFilePointer();
-      f = new LazyField(fi.name, Field.Store.YES, toRead, pointer, binary, cacheResult);
-      //Need to move the pointer ahead by toRead positions
-      fieldsStream.seek(pointer + toRead);
-    } else if (numeric != 0) {
-      f = loadNumericField(fi, numeric);
-    } else {
-      Field.Store store = Field.Store.YES;
-      Field.Index index = Field.Index.toIndex(fi.isIndexed, tokenize);
-      Field.TermVector termVector = Field.TermVector.toTermVector(fi.storeTermVector, fi.storeOffsetWithTermVector, fi.storePositionWithTermVector);
-
-      int length = fieldsStream.readVInt();
-      long pointer = fieldsStream.getFilePointer();
-      //Skip ahead of where we are by the length of what is stored
-      fieldsStream.seek(pointer+length);
-      f = new LazyField(fi.name, store, index, termVector, length, pointer, binary, cacheResult);
-    }
-    
-    f.setOmitNorms(fi.omitNorms);
-    f.setIndexOptions(fi.indexOptions);
-    doc.add(f);
-  }
-
-  private void addField(Document doc, FieldInfo fi, boolean binary, boolean tokenize, int numeric) throws CorruptIndexException, IOException {
-    final AbstractField f;
-
-    if (binary) {
-      int toRead = fieldsStream.readVInt();
-      final byte[] b = new byte[toRead];
-      fieldsStream.readBytes(b, 0, b.length);
-      f = new Field(fi.name, b);
-    } else if (numeric != 0) {
-      f = loadNumericField(fi, numeric);
-    } else {
-      Field.Index index = Field.Index.toIndex(fi.isIndexed, tokenize);
-      Field.TermVector termVector = Field.TermVector.toTermVector(fi.storeTermVector, fi.storeOffsetWithTermVector, fi.storePositionWithTermVector);
-      f = new Field(fi.name,     // name
-        fieldsStream.readString(), // read value
-        Field.Store.YES,
-        index,
-        termVector);
-    }
-    
-    f.setIndexOptions(fi.indexOptions);
-    f.setOmitNorms(fi.omitNorms);
-    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 int addFieldSize(Document doc, FieldInfo fi, boolean binary, int numeric) throws IOException {
-    final int bytesize, size;
-    switch(numeric) {
-      case 0:
-        size = fieldsStream.readVInt();
-        bytesize = binary ? size : 2*size;
-        break;
-      case FieldsWriter.FIELD_IS_NUMERIC_INT:
-      case FieldsWriter.FIELD_IS_NUMERIC_FLOAT:
-        size = bytesize = 4;
-        break;
-      case FieldsWriter.FIELD_IS_NUMERIC_LONG:
-      case FieldsWriter.FIELD_IS_NUMERIC_DOUBLE:
-        size = bytesize = 8;
-        break;
-      default:
-        throw new FieldReaderException("Invalid numeric type: " + Integer.toHexString(numeric));
-    }
-    byte[] sizebytes = new byte[4];
-    sizebytes[0] = (byte) (bytesize>>>24);
-    sizebytes[1] = (byte) (bytesize>>>16);
-    sizebytes[2] = (byte) (bytesize>>> 8);
-    sizebytes[3] = (byte)  bytesize      ;
-    doc.add(new Field(fi.name, sizebytes));
-    return size;
-  }
-
-  /**
-   * A Lazy implementation of Fieldable that defers loading of fields until asked for, instead of when the Document is
-   * loaded.
-   */
-  private class LazyField extends AbstractField implements Fieldable {
-    private int toRead;
-    private long pointer;
-    private final boolean cacheResult;
-
-    public LazyField(String name, Field.Store store, int toRead, long pointer, boolean isBinary, boolean cacheResult) {
-      super(name, store, Field.Index.NO, Field.TermVector.NO);
-      this.toRead = toRead;
-      this.pointer = pointer;
-      this.isBinary = isBinary;
-      this.cacheResult = cacheResult;
-      if (isBinary)
-        binaryLength = toRead;
-      lazy = true;
-    }
-
-    public LazyField(String name, Field.Store store, Field.Index index, Field.TermVector termVector, int toRead, long pointer, boolean isBinary, boolean cacheResult) {
-      super(name, store, index, termVector);
-      this.toRead = toRead;
-      this.pointer = pointer;
-      this.isBinary = isBinary;
-      this.cacheResult = cacheResult;
-      if (isBinary)
-        binaryLength = toRead;
-      lazy = true;
-    }
-
-    private IndexInput getFieldStream() {
-      IndexInput localFieldsStream = fieldsStreamTL.get();
-      if (localFieldsStream == null) {
-        localFieldsStream = (IndexInput) cloneableFieldsStream.clone();
-        fieldsStreamTL.set(localFieldsStream);
-      }
-      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() {
-      ensureOpen();
-      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() {
-      ensureOpen();
-      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. */
-    public String stringValue() {
-      ensureOpen();
-      if (isBinary)
-        return null;
-      else {
-        if (fieldsData == null) {
-          String result = null;
-          IndexInput localFieldsStream = getFieldStream();
-          try {
-            localFieldsStream.seek(pointer);
-            byte[] bytes = new byte[toRead];
-            localFieldsStream.readBytes(bytes, 0, toRead);
-            result = new String(bytes, "UTF-8");
-          } catch (IOException e) {
-            throw new FieldReaderException(e);
-          }
-          if (cacheResult == true){
-            fieldsData = result;
-          }
-          return result;
-        } else {
-          return (String) fieldsData;
-        }
-      }
-    }
-
-    @Override
-    public byte[] getBinaryValue(byte[] result) {
-      ensureOpen();
-
-      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;     
-    }
-  }
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FieldsWriter.java Mon Aug 29 23:13:10 2011
@@ -17,19 +17,16 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
-import java.util.List;
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.document.NumericField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 final class FieldsWriter {
-  static final int FIELD_IS_TOKENIZED = 1 << 0;
+  // NOTE: bit 0 is free here!  You can steal it!
   static final int FIELD_IS_BINARY = 1 << 1;
 
   // the old bit 1 << 2 was compressed, is now left out
@@ -114,7 +111,7 @@ final class FieldsWriter {
   void close() throws IOException {
     if (directory != null) {
       try {
-        IOUtils.closeSafely(false, fieldsStream, indexStream);
+        IOUtils.close(fieldsStream, indexStream);
       } finally {
         fieldsStream = indexStream = null;
       }
@@ -138,15 +135,17 @@ final class FieldsWriter {
     }
   }
 
-  final void writeField(int fieldNumber, Fieldable field) throws IOException {
+  final void writeField(int fieldNumber, IndexableField field) throws IOException {
     fieldsStream.writeVInt(fieldNumber);
     int bits = 0;
-    if (field.isTokenized())
-      bits |= FIELD_IS_TOKENIZED;
-    if (field.isBinary())
-      bits |= FIELD_IS_BINARY;
-    if (field instanceof NumericField) {
-      switch (((NumericField) field).getDataType()) {
+    final BytesRef bytes;
+    final String string;
+    // TODO: maybe a field should serialize itself?
+    // this way we don't bake into indexer all these
+    // specific encodings for different fields?  and apps
+    // can customize...
+    if (field.numeric()) {
+      switch (field.numericDataType()) {
         case INT:
           bits |= FIELD_IS_NUMERIC_INT; break;
         case LONG:
@@ -158,23 +157,31 @@ final class FieldsWriter {
         default:
           assert false : "Should never get here";
       }
+      string = null;
+      bytes = null;
+    } else {
+      bytes = field.binaryValue();
+      if (bytes != null) {
+        bits |= FIELD_IS_BINARY;
+        string = null;
+      } else {
+        string = field.stringValue();
+      }
     }
+
     fieldsStream.writeByte((byte) bits);
 
-    if (field.isBinary()) {
-      final byte[] data;
-      final int len;
-      final int offset;
-      data = field.getBinaryValue();
-      len = field.getBinaryLength();
-      offset =  field.getBinaryOffset();
-
-      fieldsStream.writeVInt(len);
-      fieldsStream.writeBytes(data, offset, len);
-    } else if (field instanceof NumericField) {
-      final NumericField nf = (NumericField) field;
-      final Number n = nf.getNumericValue();
-      switch (nf.getDataType()) {
+    if (bytes != null) {
+      fieldsStream.writeVInt(bytes.length);
+      fieldsStream.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+    } else if (string != null) {
+      fieldsStream.writeString(field.stringValue());
+    } else {
+      final Number n = field.numericValue();
+      if (n == null) {
+        throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
+      }
+      switch (field.numericDataType()) {
         case INT:
           fieldsStream.writeInt(n.intValue()); break;
         case LONG:
@@ -186,8 +193,6 @@ final class FieldsWriter {
         default:
           assert false : "Should never get here";
       }
-    } else {
-      fieldsStream.writeString(field.stringValue());
     }
   }
 
@@ -207,21 +212,21 @@ final class FieldsWriter {
     assert fieldsStream.getFilePointer() == position;
   }
 
-  final void addDocument(Document doc, FieldInfos fieldInfos) throws IOException {
+  final void addDocument(Iterable<? extends IndexableField> doc, FieldInfos fieldInfos) throws IOException {
     indexStream.writeLong(fieldsStream.getFilePointer());
 
     int storedCount = 0;
-    List<Fieldable> fields = doc.getFields();
-    for (Fieldable field : fields) {
-      if (field.isStored())
-          storedCount++;
+    for (IndexableField field : doc) {
+      if (field.stored()) {
+        storedCount++;
+      }
     }
     fieldsStream.writeVInt(storedCount);
 
-
-    for (Fieldable field : fields) {
-      if (field.isStored())
+    for (IndexableField field : doc) {
+      if (field.stored()) {
         writeField(fieldInfos.fieldNumber(field.name()), field);
+      }
     }
   }
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java Mon Aug 29 23:13:10 2011
@@ -17,9 +17,8 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.index.codecs.PerDocValues;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -351,9 +350,9 @@ public class FilterIndexReader extends I
   }
 
   @Override
-  public Document document(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
+  public void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
     ensureOpen();
-    return in.document(n, fieldSelector);
+    in.document(docID, visitor);
   }
 
   @Override

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Mon Aug 29 23:13:10 2011
@@ -105,7 +105,11 @@ final class FreqProxTermsWriter extends 
       }
       success = true;
     } finally {
-      IOUtils.closeSafely(!success, consumer);
+      if (success) {
+        IOUtils.close(consumer);
+      } else {
+        IOUtils.closeWhileHandlingException(consumer);
+      }
     }
   }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Mon Aug 29 23:13:10 2011
@@ -22,7 +22,6 @@ import java.util.Comparator;
 import java.util.Map;
 
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
-import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.PostingsConsumer;
@@ -82,15 +81,17 @@ final class FreqProxTermsWriterPerField 
   }
 
   @Override
-  boolean start(Fieldable[] fields, int count) {
-    for(int i=0;i<count;i++)
-      if (fields[i].isIndexed())
+  boolean start(IndexableField[] fields, int count) {
+    for(int i=0;i<count;i++) {
+      if (fields[i].indexed()) {
         return true;
+      }
+    }
     return false;
   }
 
   @Override
-  void start(Fieldable f) {
+  void start(IndexableField f) {
     if (fieldState.attributeSource.hasAttribute(PayloadAttribute.class)) {
       payloadAttribute = fieldState.attributeSource.getAttribute(PayloadAttribute.class);
     } else {

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexReader.java Mon Aug 29 23:13:10 2011
@@ -17,29 +17,28 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.search.FieldCache; // javadocs
-import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.values.IndexDocValues;
+import org.apache.lucene.search.FieldCache; // javadocs
+import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.*;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ReaderUtil;         // for javadocs
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.Closeable;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
-
 /** IndexReader is an abstract class, providing an interface for accessing an
  index.  Search of an index is done entirely through this abstract interface,
  so that any subclass which implements it is searchable.
@@ -859,7 +858,6 @@ public abstract class IndexReader implem
    * @return array of term frequency vectors. May be null if no term vectors have been
    *  stored for the specified document.
    * @throws IOException if index cannot be accessed
-   * @see org.apache.lucene.document.Field.TermVector
    */
   abstract public TermFreqVector[] getTermFreqVectors(int docNumber)
           throws IOException;
@@ -877,7 +875,6 @@ public abstract class IndexReader implem
    * @return term frequency vector May be null if field does not exist in the specified
    * document or term vector was not stored.
    * @throws IOException if index cannot be accessed
-   * @see org.apache.lucene.document.Field.TermVector
    */
   abstract public TermFreqVector getTermFreqVector(int docNumber, String field)
           throws IOException;
@@ -946,57 +943,42 @@ public abstract class IndexReader implem
     return maxDoc() - numDocs();
   }
 
+  /** Expert: visits the fields of a stored document, for
+   *  custom processing/loading of each field.  If you
+   *  simply want to load all fields, use {@link
+   *  #document(int)}.  If you want to load a subset, use
+   *  {@link DocumentStoredFieldVisitor}.  */
+  public abstract void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException;
+  
   /**
    * Returns the stored fields of the <code>n</code><sup>th</sup>
-   * <code>Document</code> in this index.
+   * <code>Document</code> in this index.  This is just
+   * sugar for using {@link DocumentStoredFieldVisitor}.
    * <p>
    * <b>NOTE:</b> for performance reasons, this method does not check if the
    * requested document is deleted, and therefore asking for a deleted document
    * may yield unspecified results. Usually this is not required, however you
    * can test if the doc is deleted by checking the {@link
    * Bits} returned from {@link MultiFields#getLiveDocs}.
+   *
+   * <b>NOTE:</b> only the content of a field is returned,
+   * if that field was stored during indexing.  Metadata
+   * like boost, omitNorm, IndexOptions, tokenized, etc.,
+   * are not preserved.
    * 
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public Document document(int n) throws CorruptIndexException, IOException {
+  // TODO: we need a separate StoredField, so that the
+  // Document returned here contains that class not
+  // IndexableField
+  public Document document(int docID) throws CorruptIndexException, IOException {
     ensureOpen();
-    return document(n, null);
+    final DocumentStoredFieldVisitor visitor = new DocumentStoredFieldVisitor();
+    document(docID, visitor);
+    return visitor.getDocument();
   }
 
-  /**
-   * Get the {@link org.apache.lucene.document.Document} at the <code>n</code>
-   * <sup>th</sup> position. The {@link FieldSelector} may be used to determine
-   * what {@link org.apache.lucene.document.Field}s to load and how they should
-   * be loaded. <b>NOTE:</b> If this Reader (more specifically, the underlying
-   * <code>FieldsReader</code>) is closed before the lazy
-   * {@link org.apache.lucene.document.Field} is loaded an exception may be
-   * thrown. If you want the value of a lazy
-   * {@link org.apache.lucene.document.Field} to be available after closing you
-   * must explicitly load it or fetch the Document again with a new loader.
-   * <p>
-   * <b>NOTE:</b> for performance reasons, this method does not check if the
-   * requested document is deleted, and therefore asking for a deleted document
-   * may yield unspecified results. Usually this is not required, however you
-   * can test if the doc is deleted by checking the {@link
-   * Bits} returned from {@link MultiFields#getLiveDocs}.
-   * 
-   * @param n Get the document at the <code>n</code><sup>th</sup> position
-   * @param fieldSelector The {@link FieldSelector} to use to determine what
-   *        Fields should be loaded on the Document. May be null, in which case
-   *        all Fields will be loaded.
-   * @return The stored fields of the
-   *         {@link org.apache.lucene.document.Document} at the nth position
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException if there is a low-level IO error
-   * @see org.apache.lucene.document.Fieldable
-   * @see org.apache.lucene.document.FieldSelector
-   * @see org.apache.lucene.document.SetBasedFieldSelector
-   * @see org.apache.lucene.document.LoadFirstFieldSelector
-   */
-  // TODO (1.5): When we convert to JDK 1.5 make this Set<String>
-  public abstract Document document(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException;
-  
   /** Returns true if any documents have been deleted */
   public abstract boolean hasDeletions();
 
@@ -1017,8 +999,8 @@ public abstract class IndexReader implem
   public abstract byte[] norms(String field) throws IOException;
 
   /** Expert: Resets the normalization factor for the named field of the named
-   * document.  By default, The norm represents the product of the field's {@link
-   * org.apache.lucene.document.Fieldable#setBoost(float) boost} and its
+   * document.  By default, the norm represents the product of the field's {@link
+   * org.apache.lucene.document.Field#setBoost(float) boost} and its
    * length normalization}.  Thus, to preserve the length normalization
    * values when resetting this, one should base the new value upon the old.
    *

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriter.java Mon Aug 29 23:13:10 2011
@@ -36,7 +36,6 @@ import java.util.concurrent.atomic.Atomi
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
 import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@@ -74,10 +73,10 @@ import org.apache.lucene.util.TwoPhaseCo
   new index if there is not already an index at the provided path
   and otherwise open the existing index.</p>
 
-  <p>In either case, documents are added with {@link #addDocument(Document)
+  <p>In either case, documents are added with {@link #addDocument(Iterable)
   addDocument} and removed with {@link #deleteDocuments(Term)} or {@link
   #deleteDocuments(Query)}. A document can be updated with {@link
-  #updateDocument(Term, Document) updateDocument} (which just deletes
+  #updateDocument(Term, Iterable) updateDocument} (which just deletes
   and then adds the entire document). When finished adding, deleting 
   and updating documents, {@link #close() close} should be called.</p>
 
@@ -1281,7 +1280,7 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void addDocument(Document doc) throws CorruptIndexException, IOException {
+  public void addDocument(Iterable<? extends IndexableField> doc) throws CorruptIndexException, IOException {
     addDocument(doc, analyzer);
   }
 
@@ -1289,7 +1288,7 @@ public class IndexWriter implements Clos
    * Adds a document to this index, using the provided analyzer instead of the
    * value of {@link #getAnalyzer()}.
    *
-   * <p>See {@link #addDocument(Document)} for details on
+   * <p>See {@link #addDocument(Iterable)} for details on
    * index and IndexWriter state after an Exception, and
    * flushing/merging temporary free space requirements.</p>
    *
@@ -1300,7 +1299,7 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void addDocument(Document doc, Analyzer analyzer) throws CorruptIndexException, IOException {
+  public void addDocument(Iterable<? extends IndexableField> doc, Analyzer analyzer) throws CorruptIndexException, IOException {
     updateDocument(null, doc, analyzer);
   }
 
@@ -1318,7 +1317,7 @@ public class IndexWriter implements Clos
    * compression), in which case you may need to fully
    * re-index your documents at that time.
    *
-   * <p>See {@link #addDocument(Document)} for details on
+   * <p>See {@link #addDocument(Iterable)} for details on
    * index and IndexWriter state after an Exception, and
    * flushing/merging temporary free space requirements.</p>
    *
@@ -1338,7 +1337,7 @@ public class IndexWriter implements Clos
    *
    * @lucene.experimental
    */
-  public void addDocuments(Iterable<Document> docs) throws CorruptIndexException, IOException {
+  public void addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws CorruptIndexException, IOException {
     addDocuments(docs, analyzer);
   }
 
@@ -1353,7 +1352,7 @@ public class IndexWriter implements Clos
    *
    * @lucene.experimental
    */
-  public void addDocuments(Iterable<Document> docs, Analyzer analyzer) throws CorruptIndexException, IOException {
+  public void addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer analyzer) throws CorruptIndexException, IOException {
     updateDocuments(null, docs, analyzer);
   }
 
@@ -1370,7 +1369,7 @@ public class IndexWriter implements Clos
    *
    * @lucene.experimental
    */
-  public void updateDocuments(Term delTerm, Iterable<Document> docs) throws CorruptIndexException, IOException {
+  public void updateDocuments(Term delTerm, Iterable<? extends Iterable<? extends IndexableField>> docs) throws CorruptIndexException, IOException {
     updateDocuments(delTerm, docs, analyzer);
   }
 
@@ -1388,7 +1387,7 @@ public class IndexWriter implements Clos
    *
    * @lucene.experimental
    */
-  public void updateDocuments(Term delTerm, Iterable<Document> docs, Analyzer analyzer) throws CorruptIndexException, IOException {
+  public void updateDocuments(Term delTerm, Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer analyzer) throws CorruptIndexException, IOException {
     ensureOpen();
     try {
       boolean success = false;
@@ -1511,7 +1510,7 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void updateDocument(Term term, Document doc) throws CorruptIndexException, IOException {
+  public void updateDocument(Term term, Iterable<? extends IndexableField> doc) throws CorruptIndexException, IOException {
     ensureOpen();
     updateDocument(term, doc, getAnalyzer());
   }
@@ -1534,7 +1533,7 @@ public class IndexWriter implements Clos
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public void updateDocument(Term term, Document doc, Analyzer analyzer)
+  public void updateDocument(Term term, Iterable<? extends IndexableField> doc, Analyzer analyzer)
       throws CorruptIndexException, IOException {
     ensureOpen();
     try {
@@ -2271,7 +2270,7 @@ public class IndexWriter implements Clos
         } catch(IOException ex) {
           prior = ex;
         } finally {
-          IOUtils.closeSafely(prior, cfsDir);
+          IOUtils.closeWhileHandlingException(prior, cfsDir);
         }
         // Perform the merge
         
@@ -2607,7 +2606,7 @@ public class IndexWriter implements Clos
         }
       }
     } finally {
-      IOUtils.closeSafely(true, cfsdir);
+      IOUtils.close(cfsdir);
     }
     
     info.dir = directory;
@@ -3034,7 +3033,7 @@ public class IndexWriter implements Clos
   DocumentsWriter getDocsWriter() {
     boolean test = false;
     assert test = true;
-    return test?docWriter: null;
+    return test ? docWriter : null;
   }
 
   /** Expert:  Return the number of documents currently

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/InvertedDocConsumerPerField.java Mon Aug 29 23:13:10 2011
@@ -19,24 +19,22 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
-import org.apache.lucene.document.Fieldable;
-
 abstract class InvertedDocConsumerPerField {
 
-  // Called once per field, and is given all Fieldable
+  // Called once per field, and is given all IndexableField
   // occurrences for this field in the document.  Return
   // true if you wish to see inverted tokens for these
   // fields:
-  abstract boolean start(Fieldable[] fields, int count) throws IOException;
+  abstract boolean start(IndexableField[] fields, int count) throws IOException;
 
   // Called before a field instance is being processed
-  abstract void start(Fieldable field);
+  abstract void start(IndexableField field);
   
   // Called once per inverted token
   abstract void add() throws IOException;
 
-  // Called once per field per document, after all Fieldable
-  // occurrences are inverted
+  // Called once per field per document, after all IndexableFields
+  // are inverted
   abstract void finish() throws IOException;
 
   // Called on hitting an aborting exception

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MultiPerDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MultiPerDocValues.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MultiPerDocValues.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MultiPerDocValues.java Mon Aug 29 23:13:10 2011
@@ -152,7 +152,7 @@ public class MultiPerDocValues extends P
   }
 
   public void close() throws IOException {
-    IOUtils.closeSafely(false, this.subs);
+    IOUtils.close(this.subs);
   }
 
   @Override

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MultiReader.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MultiReader.java Mon Aug 29 23:13:10 2011
@@ -22,8 +22,6 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -258,12 +256,11 @@ public class MultiReader extends IndexRe
     return maxDoc;
   }
 
-  // inherit javadoc
   @Override
-  public Document document(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
+  public void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
     ensureOpen();
-    int i = readerIndex(n);                          // find segment num
-    return subReaders[i].document(n - starts[i], fieldSelector);    // dispatch to segment reader
+    int i = readerIndex(docID);                          // find segment num
+    subReaders[i].document(docID - starts[i], visitor);    // dispatch to segment reader
   }
 
   @Override

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/NormsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/NormsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/NormsWriter.java Mon Aug 29 23:13:10 2011
@@ -88,7 +88,11 @@ final class NormsWriter extends Inverted
       }
       success = true;
     } finally {
-      IOUtils.closeSafely(!success, normsOut);
+      if (success) {
+        IOUtils.close(normsOut);
+      } else {
+        IOUtils.closeWhileHandlingException(normsOut);
+      }
     }
   }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/ParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/ParallelReader.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/ParallelReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/ParallelReader.java Mon Aug 29 23:13:10 2011
@@ -17,10 +17,6 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.document.FieldSelectorResult;
-import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.values.IndexDocValues;
 import org.apache.lucene.util.Bits;
@@ -351,30 +347,12 @@ public class ParallelReader extends Inde
     hasDeletions = false;
   }
 
-  // append fields from storedFieldReaders
   @Override
-  public Document document(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
+  public void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
     ensureOpen();
-    Document result = new Document();
     for (final IndexReader reader: storedFieldReaders) {
-
-      boolean include = (fieldSelector==null);
-      if (!include) {
-        Collection<String> fields = readerToFields.get(reader);
-        for (final String field : fields)
-          if (fieldSelector.accept(field) != FieldSelectorResult.NO_LOAD) {
-            include = true;
-            break;
-          }
-      }
-      if (include) {
-        List<Fieldable> fields = reader.document(n, fieldSelector).getFields();
-        for (Fieldable field : fields) {
-          result.add(field);
-        }
-      }
+      reader.document(docID, visitor);
     }
-    return result;
   }
 
   // get all vectors

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java Mon Aug 29 23:13:10 2011
@@ -75,7 +75,7 @@ final class PerFieldCodecWrapper extends
           success = true;
         } finally {
           if (!success) {
-            IOUtils.closeSafely(true, consumers);
+            IOUtils.closeWhileHandlingException(consumers);
           }
         }
       }
@@ -90,7 +90,7 @@ final class PerFieldCodecWrapper extends
 
     @Override
     public void close() throws IOException {
-      IOUtils.closeSafely(false, consumers);
+      IOUtils.close(consumers);
     }
   }
 
@@ -123,7 +123,7 @@ final class PerFieldCodecWrapper extends
           // If we hit exception (eg, IOE because writer was
           // committing, or, for any other reason) we must
           // go back and close all FieldsProducers we opened:
-          IOUtils.closeSafely(true, producers.values());
+          IOUtils.closeWhileHandlingException(producers.values());
         }
       }
     }
@@ -172,7 +172,7 @@ final class PerFieldCodecWrapper extends
     
     @Override
     public void close() throws IOException {
-      IOUtils.closeSafely(false, codecs.values());
+      IOUtils.close(codecs.values());
     }
   }
 
@@ -230,7 +230,7 @@ final class PerFieldCodecWrapper extends
         success = true;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, producers.values());
+          IOUtils.closeWhileHandlingException(producers.values());
         }
       }
     }
@@ -249,7 +249,7 @@ final class PerFieldCodecWrapper extends
     }
     
     public void close() throws IOException {
-      IOUtils.closeSafely(false, codecs.values());
+      IOUtils.close(codecs.values());
     }
   }
   
@@ -266,7 +266,7 @@ final class PerFieldCodecWrapper extends
     }
 
     public void close() throws IOException {
-      IOUtils.closeSafely(false, consumers);
+      IOUtils.close(consumers);
     }
 
     @Override

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java Mon Aug 29 23:13:10 2011
@@ -25,9 +25,7 @@ import java.util.Map.Entry;
 
 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.FieldType;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.LockObtainFailedException;
@@ -71,12 +69,11 @@ public class PersistentSnapshotDeletionP
       // index is allowed to have exactly one document or 0.
       if (numDocs == 1) {
         Document doc = r.document(r.maxDoc() - 1);
-        Field sid = doc.getField(SNAPSHOTS_ID);
-        if (sid == null) {
+        if (doc.getField(SNAPSHOTS_ID) == null) {
           throw new IllegalStateException("directory is not a valid snapshots store!");
         }
         doc.removeField(SNAPSHOTS_ID);
-        for (Fieldable f : doc.getFields()) {
+        for (IndexableField f : doc) {
           snapshots.put(f.name(), f.stringValue());
         }
       } else if (numDocs != 0) {
@@ -189,12 +186,14 @@ public class PersistentSnapshotDeletionP
   private void persistSnapshotInfos(String id, String segment) throws IOException {
     writer.deleteAll();
     Document d = new Document();
-    d.add(new Field(SNAPSHOTS_ID, "", Store.YES, Index.NO));
+    FieldType ft = new FieldType();
+    ft.setStored(true);
+    d.add(new Field(SNAPSHOTS_ID, ft, ""));
     for (Entry<String, String> e : super.getSnapshots().entrySet()) {
-      d.add(new Field(e.getKey(), e.getValue(), Store.YES, Index.NO));
+      d.add(new Field(e.getKey(), ft, e.getValue()));
     }
     if (id != null) {
-      d.add(new Field(id, segment, Store.YES, Index.NO));
+      d.add(new Field(id, ft, segment));
     }
     writer.addDocument(d);
     writer.commit();

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java Mon Aug 29 23:13:10 2011
@@ -122,7 +122,7 @@ final class SegmentCoreReaders {
   
   synchronized void decRef() throws IOException {
     if (ref.decrementAndGet() == 0) {
-      IOUtils.closeSafely(false, fields, perDocProducer, termVectorsReaderOrig,
+      IOUtils.close(fields, perDocProducer, termVectorsReaderOrig,
           fieldsReaderOrig, cfsReader, storeCFSReader);
       // Now, notify any ReaderFinished listeners:
       if (owner != null) {

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Mon Aug 29 23:13:10 2011
@@ -331,7 +331,7 @@ public final class SegmentInfos implemen
       if (!success) {
         // We hit an exception above; try to close the file
         // but suppress any exception:
-        IOUtils.closeSafely(true, segnOutput);
+        IOUtils.closeWhileHandlingException(segnOutput);
         try {
           // Try not to leave a truncated segments_N file in
           // the index:

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon Aug 29 23:13:10 2011
@@ -335,6 +335,10 @@ final class SegmentMerger {
           // skip deleted docs
           continue;
         }
+        // TODO: this could be more efficient using
+        // FieldVisitor instead of loading/writing entire
+        // doc; ie we just have to renumber the field number
+        // on the fly?
         // NOTE: it's very important to first assign to doc then pass it to
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
         Document doc = reader.reader.document(j);
@@ -569,7 +573,11 @@ final class SegmentMerger {
                                      slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
       success = true;
     } finally {
-      IOUtils.closeSafely(!success, consumer);
+      if (success) {
+        IOUtils.close(consumer);
+      } else {
+        IOUtils.closeWhileHandlingException(consumer);
+      }
     }
   }
 
@@ -598,7 +606,11 @@ final class SegmentMerger {
         docsConsumer.merge(mergeState, multiPerDocValues);
         success = true;
       } finally {
-        IOUtils.closeSafely(!success, docsConsumer);
+        if (success) {
+          IOUtils.close(docsConsumer);
+        } else {
+          IOUtils.closeWhileHandlingException(docsConsumer);
+        }
       }
     }
     /* don't close the perDocProducers here since they are private segment producers
@@ -650,7 +662,11 @@ final class SegmentMerger {
       }
       success = true;
     } finally {
-      IOUtils.closeSafely(!success, output);
+      if (success) {
+        IOUtils.close(output);
+      } else {
+        IOUtils.closeWhileHandlingException(output);
+      }
     }
   }
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon Aug 29 23:13:10 2011
@@ -27,13 +27,11 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.FieldSelector;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.codecs.PerDocValues;
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BitVector;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -455,10 +453,9 @@ public class SegmentReader extends Index
     return core.fieldInfos;
   }
 
-  @Override
-  public Document document(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
+  public void document(int docID, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
     ensureOpen();
-    return getFieldsReader().doc(n, fieldSelector);
+    getFieldsReader().visitDocument(docID, visitor);
   }
 
   @Override

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Mon Aug 29 23:13:10 2011
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
-import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
@@ -41,12 +40,12 @@ final class StoredFieldsWriter {
   }
 
   private int numStoredFields;
-  private Fieldable[] storedFields;
+  private IndexableField[] storedFields;
   private int[] fieldNumbers;
 
   public void reset() {
     numStoredFields = 0;
-    storedFields = new Fieldable[1];
+    storedFields = new IndexableField[1];
     fieldNumbers = new int[1];
   }
 
@@ -123,10 +122,10 @@ final class StoredFieldsWriter {
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
   }
 
-  public void addField(Fieldable field, FieldInfo fieldInfo) throws IOException {
+  public void addField(IndexableField field, FieldInfo fieldInfo) throws IOException {
     if (numStoredFields == storedFields.length) {
       int newSize = ArrayUtil.oversize(numStoredFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
-      Fieldable[] newArray = new Fieldable[newSize];
+      IndexableField[] newArray = new IndexableField[newSize];
       System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
       storedFields = newArray;
     }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermFreqVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermFreqVector.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermFreqVector.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermFreqVector.java Mon Aug 29 23:13:10 2011
@@ -26,7 +26,7 @@ import org.apache.lucene.util.BytesRef;
  */
 public interface TermFreqVector {
   /**
-   * The {@link org.apache.lucene.document.Fieldable} name. 
+   * The {@link org.apache.lucene.index.IndexableField} name. 
    * @return The name of the field this vector is associated with.
    * 
    */ 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java Mon Aug 29 23:13:10 2011
@@ -193,7 +193,7 @@ class TermVectorsReader implements Clone
   }
 
   public void close() throws IOException {
-    IOUtils.closeSafely(false, tvx, tvd, tvf);
+    IOUtils.close(tvx, tvd, tvf);
   }
 
   /**

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Mon Aug 29 23:13:10 2011
@@ -57,7 +57,7 @@ final class TermVectorsTermsWriter exten
       fill(state.numDocs);
       assert state.segmentName != null;
       String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
-      IOUtils.closeSafely(false, tvx, tvf, tvd);
+      IOUtils.close(tvx, tvf, tvd);
       tvx = tvd = tvf = null;
       if (4+((long) state.numDocs)*16 != state.directory.fileLength(idxName)) {
         throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
@@ -107,7 +107,7 @@ final class TermVectorsTermsWriter exten
         success = true;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, tvx, tvd, tvf);
+          IOUtils.closeWhileHandlingException(tvx, tvd, tvf);
         }
       }
 
@@ -161,7 +161,7 @@ final class TermVectorsTermsWriter exten
   public void abort() {
     hasVectors = false;
     try {
-      IOUtils.closeSafely(true, tvx, tvd, tvf);
+      IOUtils.closeWhileHandlingException(tvx, tvd, tvf);
     } catch (IOException e) {
       // cannot happen since we suppress exceptions
       throw new RuntimeException(e);

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java Mon Aug 29 23:13:10 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
-import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
@@ -55,17 +54,17 @@ final class TermVectorsTermsWriterPerFie
   }
 
   @Override
-  boolean start(Fieldable[] fields, int count) {
+  boolean start(IndexableField[] fields, int count) {
     doVectors = false;
     doVectorPositions = false;
     doVectorOffsets = false;
 
     for(int i=0;i<count;i++) {
-      Fieldable field = fields[i];
-      if (field.isIndexed() && field.isTermVectorStored()) {
+      IndexableField field = fields[i];
+      if (field.indexed() && field.storeTermVectors()) {
         doVectors = true;
-        doVectorPositions |= field.isStorePositionWithTermVector();
-        doVectorOffsets |= field.isStoreOffsetWithTermVector();
+        doVectorPositions |= field.storeTermVectorPositions();
+        doVectorOffsets |= field.storeTermVectorOffsets();
       }
     }
 
@@ -188,7 +187,7 @@ final class TermVectorsTermsWriterPerFie
   }
 
   @Override
-  void start(Fieldable f) {
+  void start(IndexableField f) {
     if (doVectorOffsets) {
       offsetAttribute = fieldState.attributeSource.addAttribute(OffsetAttribute.class);
     } else {

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java Mon Aug 29 23:13:10 2011
@@ -45,7 +45,7 @@ final class TermVectorsWriter {
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeSafely(true, tvx, tvd, tvf);
+        IOUtils.closeWhileHandlingException(tvx, tvd, tvf);
       }
     }
 
@@ -194,6 +194,6 @@ final class TermVectorsWriter {
   final void close() throws IOException {
     // make an effort to close all streams we can but remember and re-throw
     // the first exception encountered in this process
-    IOUtils.closeSafely(false, tvx, tvd, tvf);
+    IOUtils.close(tvx, tvd, tvf);
   }
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermsHashConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermsHashConsumerPerField.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermsHashConsumerPerField.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermsHashConsumerPerField.java Mon Aug 29 23:13:10 2011
@@ -24,13 +24,11 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
-import org.apache.lucene.document.Fieldable;
-
 abstract class TermsHashConsumerPerField {
-  abstract boolean start(Fieldable[] fields, int count) throws IOException;
+  abstract boolean start(IndexableField[] fields, int count) throws IOException;
   abstract void finish() throws IOException;
   abstract void skippingLongTerm() throws IOException;
-  abstract void start(Fieldable field);
+  abstract void start(IndexableField field);
   abstract void newTerm(int termID) throws IOException;
   abstract void addTerm(int termID) throws IOException;
   abstract int getStreamCount();

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java Mon Aug 29 23:13:10 2011
@@ -22,7 +22,6 @@ import java.util.Comparator;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
@@ -116,7 +115,7 @@ final class TermsHashPerField extends In
   private boolean doNextCall;
 
   @Override
-  void start(Fieldable f) {
+  void start(IndexableField f) {
     termAtt = fieldState.attributeSource.getAttribute(TermToBytesRefAttribute.class);
     termBytesRef = termAtt.getBytesRef();
     consumer.start(f);
@@ -126,11 +125,12 @@ final class TermsHashPerField extends In
   }
 
   @Override
-  boolean start(Fieldable[] fields, int count) throws IOException {
+  boolean start(IndexableField[] fields, int count) throws IOException {
     doCall = consumer.start(fields, count);
     bytesHash.reinit();
-    if (nextPerField != null)
+    if (nextPerField != null) {
       doNextCall = nextPerField.start(fields, count);
+    }
     return doCall || doNextCall;
   }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java Mon Aug 29 23:13:10 2011
@@ -88,7 +88,7 @@ public class BlockTermsWriter extends Fi
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeSafely(true, out);
+        IOUtils.closeWhileHandlingException(out);
       }
     }
   }
@@ -138,7 +138,7 @@ public class BlockTermsWriter extends Fi
       }
       writeTrailer(dirStart);
     } finally {
-      IOUtils.closeSafely(false, out, postingsWriter, termsIndexWriter);
+      IOUtils.close(out, postingsWriter, termsIndexWriter);
     }
   }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java Mon Aug 29 23:13:10 2011
@@ -160,7 +160,7 @@ public class BlockTreeTermsReader extend
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeSafely(true, indexIn, this);
+        IOUtils.closeWhileHandlingException(indexIn, this);
       } else if (indexDivisor != -1) {
         indexIn.close();
       }
@@ -194,7 +194,7 @@ public class BlockTreeTermsReader extend
   @Override
   public void close() throws IOException {
     try {
-      IOUtils.closeSafely(false, in, postingsReader);
+      IOUtils.close(in, postingsReader);
     } finally { 
       for(FieldReader field : fields.values()) {
         field.close();

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsWriter.java Mon Aug 29 23:13:10 2011
@@ -171,7 +171,7 @@ public class BlockTreeTermsWriter extend
       success = true;
     } finally {
       if (!success) {
-        IOUtils.closeSafely(true, out, indexOut);
+        IOUtils.closeWhileHandlingException(out, indexOut);
       }
     }
     this.indexOut = indexOut;
@@ -937,7 +937,7 @@ public class BlockTreeTermsWriter extend
     } catch (IOException ioe2) {
       ioe = ioe2;
     } finally {
-      IOUtils.closeSafely(ioe, out, indexOut, postingsWriter);
+      IOUtils.closeWhileHandlingException(ioe, out, indexOut, postingsWriter);
     }
   }
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/Codec.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/Codec.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/Codec.java Mon Aug 29 23:13:10 2011
@@ -34,12 +34,22 @@ public abstract class Codec {
   /** Unique name that's used to retrieve this codec when
    *  reading the index */
   public final String name;
-  private boolean dvUseCompoundFile = true;
-  private Comparator<BytesRef> docValuesSortComparator = BytesRef
-      .getUTF8SortedAsUnicodeComparator();
+  protected final boolean dvUseCompoundFile;
+  protected final Comparator<BytesRef> docValuesSortComparator;
   
   protected Codec(String name) {
+    this(name, true);
+  }
+  
+  protected Codec(String name, boolean docValuesUseCompoundFile) {
+    this(name, docValuesUseCompoundFile, BytesRef.getUTF8SortedAsUnicodeComparator());
+  }
+
+  protected Codec(String name, boolean docValuesUseCompoundFile,
+      Comparator<BytesRef> docValuesSortComparator) {
     this.name = name;
+    this.dvUseCompoundFile = docValuesUseCompoundFile;
+    this.docValuesSortComparator = docValuesSortComparator;
   }
 
   /** Writes a new segment */
@@ -77,43 +87,34 @@ public abstract class Codec {
 
   /** Records all file extensions this codec uses */
   public abstract void getExtensions(Set<String> extensions);
-  
-
-  /**
-   * If set to <code>true</code> this codec will use a compound file for
-   * IndexDocValues, otherwise each IndexDocValues field will create up to 2
-   * files per segment.
-   * <p>
-   * NOTE: The default values is <code>true</code>.
-   */
-  public void setDocValuesUseCFS(boolean docValuesUseCFS) {
-    this.dvUseCompoundFile = docValuesUseCFS;
-  }
 
   /**
    * Returns <code>true</code> iff compound file should be used for
-   * IndexDocValues, otherwise <code>false</code>.
+   * IndexDocValues, otherwise <code>false</code>. The default is
+   * <code>true</code>.
+   * <p>
+   * NOTE: To change the default value you need to subclass a {@link Codec} with
+   * a distinct name since this value is final and should not be changed to
+   * prevent the risk of a index corruption. This setting is private to a
+   * {@link Codec}. If you intend to change this value on an existing
+   * {@link Codec} re-indexing is required.
    * 
-   * @see #setDocValuesUseCFS(boolean)
    * @return <code>true</code> iff compound file should be used for
    *         IndexDocValues, otherwise <code>false</code>.
    */
   public boolean getDocValuesUseCFS() {
     return dvUseCompoundFile;
   }
-  
-  /**
-   * Sets the {@link BytesRef} comparator for sorted IndexDocValue variants. The
-   * default is {@link BytesRef#getUTF8SortedAsUnicodeComparator()}. *
-   */
-  public void setDocValuesSortComparator(
-      Comparator<BytesRef> docValuesSortComparator) {
-    this.docValuesSortComparator = docValuesSortComparator;
-  }
 
   /**
    * Returns the {@link BytesRef} comparator for sorted IndexDocValue variants.
    * The default is {@link BytesRef#getUTF8SortedAsUnicodeComparator()}.
+   * <p>
+   * NOTE: To change the default value you need to subclass a {@link Codec} with
+   * a distinct name since this value is final and should not be changed to
+   * prevent the risk of a index corruption. This setting is private to a
+   * {@link Codec}. If you intend to change this value on an existing
+   * {@link Codec} re-indexing is required.
    */
   public Comparator<BytesRef> getDocValuesSortComparator() {
     return docValuesSortComparator;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java Mon Aug 29 23:13:10 2011
@@ -189,7 +189,7 @@ public class DefaultDocValuesProducer ex
     } else {
       toClose = closeables;
     } 
-    IOUtils.closeSafely(false, toClose);
+    IOUtils.close(toClose);
   }
 
   @Override

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

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java?rev=1163047&r1=1163046&r2=1163047&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java Mon Aug 29 23:13:10 2011
@@ -111,7 +111,9 @@ public class FixedGapTermsIndexReader ex
       }
       success = true;
     } finally {
-      if (!success) IOUtils.closeSafely(true, in);
+      if (!success) {
+        IOUtils.closeWhileHandlingException(in);
+      }
       if (indexDivisor > 0) {
         in.close();
         in = null;

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

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