lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sim...@apache.org
Subject svn commit: r1388574 [11/45] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/eclipse/dot.settings/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/lucene/anal...
Date Fri, 21 Sep 2012 17:22:27 GMT
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java Fri Sep 21 17:21:34 2012
@@ -19,13 +19,13 @@ package org.apache.lucene.codecs;
 import java.io.Closeable;
 import java.io.IOException;
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.StorableField;
+import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.index.AtomicReader;
 
 /**
  * Codec API for writing stored fields:
@@ -33,7 +33,7 @@ import org.apache.lucene.util.Bits;
  * <ol>
  *   <li>For every document, {@link #startDocument(int)} is called,
  *       informing the Codec how many fields will be written.
- *   <li>{@link #writeField(FieldInfo, IndexableField)} is called for 
+ *   <li>{@link #writeField(FieldInfo, StorableField)} is called for 
  *       each field in the document.
  *   <li>After all documents have been written, {@link #finish(FieldInfos, int)} 
  *       is called for verification/sanity-checks.
@@ -45,14 +45,14 @@ import org.apache.lucene.util.Bits;
 public abstract class StoredFieldsWriter implements Closeable {
   
   /** Called before writing the stored fields of the document.
-   *  {@link #writeField(FieldInfo, IndexableField)} will be called
+   *  {@link #writeField(FieldInfo, StorableField)} will be called
    *  <code>numStoredFields</code> times. Note that this is
    *  called even if the document has no stored fields, in
    *  this case <code>numStoredFields</code> will be zero. */
   public abstract void startDocument(int numStoredFields) throws IOException;
   
   /** Writes a single stored field. */
-  public abstract void writeField(FieldInfo info, IndexableField field) throws IOException;
+  public abstract void writeField(FieldInfo info, StorableField field) throws IOException;
 
   /** Aborts writing entirely, implementation should remove
    *  any partially-written files, etc. */
@@ -69,7 +69,7 @@ public abstract class StoredFieldsWriter
   /** Merges in the stored fields from the readers in 
    *  <code>mergeState</code>. The default implementation skips
    *  over deleted documents, and uses {@link #startDocument(int)},
-   *  {@link #writeField(FieldInfo, IndexableField)}, and {@link #finish(FieldInfos, int)},
+   *  {@link #writeField(FieldInfo, StorableField)}, and {@link #finish(FieldInfos, int)},
    *  returning the number of documents that were written.
    *  Implementations can override this method for more sophisticated
    *  merging (bulk-byte copying, etc). */
@@ -89,7 +89,7 @@ public abstract class StoredFieldsWriter
         // on the fly?
         // NOTE: it's very important to first assign to doc then pass it to
         // fieldsWriter.addDocument; see LUCENE-1282
-        Document doc = reader.document(i);
+        StoredDocument doc = reader.document(i);
         addDocument(doc, mergeState.fieldInfos);
         docCount++;
         mergeState.checkAbort.work(300);
@@ -100,20 +100,16 @@ public abstract class StoredFieldsWriter
   }
   
   /** sugar method for startDocument() + writeField() for every stored field in the document */
-  protected final void addDocument(Iterable<? extends IndexableField> doc, FieldInfos fieldInfos) throws IOException {
+  protected final void addDocument(Iterable<? extends StorableField> doc, FieldInfos fieldInfos) throws IOException {
     int storedCount = 0;
-    for (IndexableField field : doc) {
-      if (field.fieldType().stored()) {
-        storedCount++;
-      }
+    for (StorableField field : doc) {
+      storedCount++;
     }
     
     startDocument(storedCount);
 
-    for (IndexableField field : doc) {
-      if (field.fieldType().stored()) {
+    for (StorableField field : doc) {
         writeField(fieldInfos.fieldInfo(field.name()), field);
-      }
     }
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java Fri Sep 21 17:21:34 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.Iterator;
 
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
@@ -27,8 +28,6 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
-import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -170,12 +169,7 @@ public abstract class TermVectorsWriter 
       final AtomicReader reader = mergeState.readers.get(i);
       final int maxDoc = reader.maxDoc();
       final Bits liveDocs = reader.getLiveDocs();
-      // set PayloadProcessor
-      if (mergeState.payloadProcessorProvider != null) {
-        mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
-      } else {
-        mergeState.currentReaderPayloadProcessor = null;
-      }
+
       for (int docID = 0; docID < maxDoc; docID++) {
         if (liveDocs != null && !liveDocs.get(docID)) {
           // skip deleted docs
@@ -194,19 +188,21 @@ public abstract class TermVectorsWriter 
   }
   
   /** Safe (but, slowish) default method to write every
-   *  vector field in the document.  This default
-   *  implementation requires that the vectors implement
-   *  both Fields.size and
-   *  Terms.size. */
+   *  vector field in the document. */
   protected final void addAllDocVectors(Fields vectors, MergeState mergeState) throws IOException {
     if (vectors == null) {
       startDocument(0);
       return;
     }
 
-    final int numFields = vectors.size();
+    int numFields = vectors.size();
     if (numFields == -1) {
-      throw new IllegalStateException("vectors.size() must be implemented (it returned -1)");
+      // count manually! TODO: Maybe enforce that Fields.size() returns something valid?
+      numFields = 0;
+      for (final Iterator<String> it = vectors.iterator(); it.hasNext(); ) {
+        it.next();
+        numFields++;
+      }
     }
     startDocument(numFields);
     
@@ -215,10 +211,9 @@ public abstract class TermVectorsWriter 
     TermsEnum termsEnum = null;
     DocsAndPositionsEnum docsAndPositionsEnum = null;
     
-    final ReaderPayloadProcessor readerPayloadProcessor = mergeState.currentReaderPayloadProcessor;
-    PayloadProcessor payloadProcessor = null;
-
+    int fieldCount = 0;
     for(String fieldName : vectors) {
+      fieldCount++;
       final FieldInfo fieldInfo = mergeState.fieldInfos.fieldInfo(fieldName);
 
       assert lastFieldName == null || fieldName.compareTo(lastFieldName) > 0: "lastFieldName=" + lastFieldName + " fieldName=" + fieldName;
@@ -235,9 +230,14 @@ public abstract class TermVectorsWriter 
       final boolean hasPayloads = terms.hasPayloads();
       assert !hasPayloads || hasPositions;
       
-      final int numTerms = (int) terms.size();
+      int numTerms = (int) terms.size();
       if (numTerms == -1) {
-        throw new IllegalStateException("terms.size() must be implemented (it returned -1)");
+        // count manually. It is stupid, but needed, as Terms.size() is not a mandatory statistics function
+        numTerms = 0;
+        termsEnum = terms.iterator(termsEnum);
+        while(termsEnum.next() != null) {
+          numTerms++;
+        }
       }
       
       startField(fieldInfo, numTerms, hasPositions, hasOffsets, hasPayloads);
@@ -250,10 +250,6 @@ public abstract class TermVectorsWriter 
         final int freq = (int) termsEnum.totalTermFreq();
         
         startTerm(termsEnum.term(), freq);
-        
-        if (hasPayloads && readerPayloadProcessor != null) {
-          payloadProcessor = readerPayloadProcessor.getProcessor(fieldName, termsEnum.term());
-        }
 
         if (hasPositions || hasOffsets) {
           docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum);
@@ -268,17 +264,7 @@ public abstract class TermVectorsWriter 
             final int startOffset = docsAndPositionsEnum.startOffset();
             final int endOffset = docsAndPositionsEnum.endOffset();
             
-            BytesRef payload = docsAndPositionsEnum.getPayload();
-                
-            if (payloadProcessor != null && payload != null) {
-              // to not violate the D&P api, we must give the processor a private copy
-              payload = BytesRef.deepCopyOf(payload);
-              payloadProcessor.processPayload(payload);
-              if (payload.length == 0) {
-                // don't let PayloadProcessors corrumpt the index
-                payload = null;
-              }
-            }
+            final BytesRef payload = docsAndPositionsEnum.getPayload();
 
             assert !hasPositions || pos >= 0;
             addPosition(pos, startOffset, endOffset, payload);
@@ -287,6 +273,7 @@ public abstract class TermVectorsWriter 
       }
       assert termCount == numTerms;
     }
+    assert fieldCount == numFields;
   }
   
   /** Return the BytesRef Comparator used to sort terms

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermsConsumer.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermsConsumer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermsConsumer.java Fri Sep 21 17:21:34 2012
@@ -154,14 +154,7 @@ public abstract class TermsConsumer {
         postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, DocsAndPositionsEnum.FLAG_PAYLOADS);
         assert postingsEnumIn != null;
         postingsEnum.reset(postingsEnumIn);
-        // set PayloadProcessor
-        if (mergeState.payloadProcessorProvider != null) {
-          for (int i = 0; i < mergeState.readers.size(); i++) {
-            if (mergeState.readerPayloadProcessor[i] != null) {
-              mergeState.currentPayloadProcessor[i] = mergeState.readerPayloadProcessor[i].getProcessor(mergeState.fieldInfo.name, term);
-            }
-          }
-        }
+
         final PostingsConsumer postingsConsumer = startTerm(term);
         final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs);
         if (stats.docFreq > 0) {
@@ -188,14 +181,7 @@ public abstract class TermsConsumer {
         postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn);
         assert postingsEnumIn != null;
         postingsEnum.reset(postingsEnumIn);
-        // set PayloadProcessor
-        if (mergeState.payloadProcessorProvider != null) {
-          for (int i = 0; i < mergeState.readers.size(); i++) {
-            if (mergeState.readerPayloadProcessor[i] != null) {
-              mergeState.currentPayloadProcessor[i] = mergeState.readerPayloadProcessor[i].getProcessor(mergeState.fieldInfo.name, term);
-            }
-          }
-        }
+
         final PostingsConsumer postingsConsumer = startTerm(term);
         final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs);
         if (stats.docFreq > 0) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java Fri Sep 21 17:21:34 2012
@@ -163,7 +163,7 @@ final class BitVector implements Cloneab
       int c = 0;
       int end = bits.length;
       for (int i = 0; i < end; i++) {
-        c += BYTE_COUNTS[bits[i] & 0xFF];	  // sum bits per byte
+        c += BYTE_COUNTS[bits[i] & 0xFF];  // sum bits per byte
       }
       count = c;
     }
@@ -176,12 +176,12 @@ final class BitVector implements Cloneab
     int c = 0;
     int end = bits.length;
     for (int i = 0; i < end; i++) {
-      c += BYTE_COUNTS[bits[i] & 0xFF];	  // sum bits per byte
+      c += BYTE_COUNTS[bits[i] & 0xFF];  // sum bits per byte
     }
     return c;
   }
 
-  private static final byte[] BYTE_COUNTS = {	  // table of bits/byte
+  private static final byte[] BYTE_COUNTS = {  // table of bits/byte
     0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
     1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
     1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java Fri Sep 21 17:21:34 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene4
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -30,7 +31,10 @@ import org.apache.lucene.codecs.perfield
 
 /**
  * Implements the Lucene 4.0 index format, with configurable per-field postings formats.
- * 
+ * <p>
+ * If you want to reuse functionality of this codec in another codec, extend
+ * {@link FilterCodec}.
+ *
  * @see org.apache.lucene.codecs.lucene40 package documentation for file format details.
  * @lucene.experimental
  */
@@ -58,42 +62,42 @@ public class Lucene40Codec extends Codec
   }
   
   @Override
-  public StoredFieldsFormat storedFieldsFormat() {
+  public final StoredFieldsFormat storedFieldsFormat() {
     return fieldsFormat;
   }
   
   @Override
-  public TermVectorsFormat termVectorsFormat() {
+  public final TermVectorsFormat termVectorsFormat() {
     return vectorsFormat;
   }
 
   @Override
-  public DocValuesFormat docValuesFormat() {
+  public final DocValuesFormat docValuesFormat() {
     return docValuesFormat;
   }
 
   @Override
-  public PostingsFormat postingsFormat() {
+  public final PostingsFormat postingsFormat() {
     return postingsFormat;
   }
   
   @Override
-  public FieldInfosFormat fieldInfosFormat() {
+  public final FieldInfosFormat fieldInfosFormat() {
     return fieldInfosFormat;
   }
   
   @Override
-  public SegmentInfoFormat segmentInfoFormat() {
+  public final SegmentInfoFormat segmentInfoFormat() {
     return infosFormat;
   }
 
   @Override
-  public NormsFormat normsFormat() {
+  public final NormsFormat normsFormat() {
     return normsFormat;
   }
   
   @Override
-  public LiveDocsFormat liveDocsFormat() {
+  public final LiveDocsFormat liveDocsFormat() {
     return liveDocsFormat;
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java Fri Sep 21 17:21:34 2012
@@ -100,8 +100,7 @@ public class Lucene40FieldInfosReader ex
     }
   }
 
-  public DocValues.Type getDocValuesType(
-      final byte b) {
+  private static DocValues.Type getDocValuesType(final byte b) {
     switch(b) {
       case 0:
         return null;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java Fri Sep 21 17:21:34 2012
@@ -159,7 +159,7 @@ import org.apache.lucene.util.fst.FST; /
  * with the frequency of the term in that document (except when frequencies are
  * omitted: {@link IndexOptions#DOCS_ONLY}).</p>
  * <ul>
- *   <li>FreqFile (.frq) --&gt; Header, &lt;TermFreqs, SkipData&gt; <sup>TermCount</sup></li>
+ *   <li>FreqFile (.frq) --&gt; Header, &lt;TermFreqs, SkipData?&gt; <sup>TermCount</sup></li>
  *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *   <li>TermFreqs --&gt; &lt;TermFreq&gt; <sup>DocFreq</sup></li>
  *   <li>TermFreq --&gt; DocDelta[, Freq?]</li>
@@ -269,7 +269,7 @@ import org.apache.lucene.util.fst.FST; /
 // TODO: this class could be created by wrapping
 // BlockTreeTermsDict around Lucene40PostingsBaseFormat; ie
 // we should not duplicate the code from that class here:
-public class Lucene40PostingsFormat extends PostingsFormat {
+public final class Lucene40PostingsFormat extends PostingsFormat {
 
   private final int minBlockSize;
   private final int maxBlockSize;
@@ -314,7 +314,7 @@ public class Lucene40PostingsFormat exte
       FieldsProducer ret = new BlockTreeTermsReader(
                                                     state.dir,
                                                     state.fieldInfos,
-                                                    state.segmentInfo.name,
+                                                    state.segmentInfo,
                                                     postings,
                                                     state.context,
                                                     state.segmentSuffix,

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java Fri Sep 21 17:21:34 2012
@@ -325,7 +325,7 @@ public class Lucene40PostingsReader exte
     
     SegmentDocsEnumBase(IndexInput startFreqIn, Bits liveDocs) {
       this.startFreqIn = startFreqIn;
-      this.freqIn = (IndexInput)startFreqIn.clone();
+      this.freqIn = startFreqIn.clone();
       this.liveDocs = liveDocs;
       
     }
@@ -474,7 +474,7 @@ public class Lucene40PostingsReader exte
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
-          skipper = new Lucene40SkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+          skipper = new Lucene40SkipListReader(freqIn.clone(), maxSkipLevels, skipInterval);
         }
 
         if (!skipped) {
@@ -705,8 +705,8 @@ public class Lucene40PostingsReader exte
 
     public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) {
       startFreqIn = freqIn;
-      this.freqIn = (IndexInput) freqIn.clone();
-      this.proxIn = (IndexInput) proxIn.clone();
+      this.freqIn = freqIn.clone();
+      this.proxIn = proxIn.clone();
     }
 
     public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
@@ -795,7 +795,7 @@ public class Lucene40PostingsReader exte
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
-          skipper = new Lucene40SkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+          skipper = new Lucene40SkipListReader(freqIn.clone(), maxSkipLevels, skipInterval);
         }
 
         if (!skipped) {
@@ -913,8 +913,8 @@ public class Lucene40PostingsReader exte
 
     public SegmentFullPositionsEnum(IndexInput freqIn, IndexInput proxIn) {
       startFreqIn = freqIn;
-      this.freqIn = (IndexInput) freqIn.clone();
-      this.proxIn = (IndexInput) proxIn.clone();
+      this.freqIn = freqIn.clone();
+      this.proxIn = proxIn.clone();
     }
 
     public SegmentFullPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
@@ -1009,7 +1009,7 @@ public class Lucene40PostingsReader exte
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
-          skipper = new Lucene40SkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+          skipper = new Lucene40SkipListReader(freqIn.clone(), maxSkipLevels, skipInterval);
         }
 
         if (!skipped) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java Fri Sep 21 17:21:34 2012
@@ -185,8 +185,6 @@ public final class Lucene40PostingsWrite
   int lastDocID;
   int df;
   
-  /** Adds a new doc in this term.  If this returns null
-   *  then we just skip consuming positions/payloads. */
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
     // if (DEBUG) System.out.println("SPW:   startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq + " freqOut.fp=" + freqOut.getFilePointer());

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoReader.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfoReader.java Fri Sep 21 17:21:34 2012
@@ -24,6 +24,7 @@ import java.util.Set;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.SegmentInfoReader;
+import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
@@ -50,10 +51,17 @@ public class Lucene40SegmentInfoReader e
                                    Lucene40SegmentInfoFormat.VERSION_CURRENT);
       final String version = input.readString();
       final int docCount = input.readInt();
+      if (docCount < 0) {
+        throw new CorruptIndexException("invalid docCount: " + docCount + " (resource=" + input + ")");
+      }
       final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
       final Map<String,String> diagnostics = input.readStringStringMap();
       final Map<String,String> attributes = input.readStringStringMap();
       final Set<String> files = input.readStringSet();
+      
+      if (input.getFilePointer() != input.length()) {
+        throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
+      }
 
       final SegmentInfo si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile,
                                              null, diagnostics, Collections.unmodifiableMap(attributes));

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsReader.java Fri Sep 21 17:21:34 2012
@@ -61,7 +61,7 @@ public final class Lucene40StoredFieldsR
   @Override
   public Lucene40StoredFieldsReader clone() {
     ensureOpen();
-    return new Lucene40StoredFieldsReader(fieldInfos, numTotalDocs, size, (IndexInput)fieldsStream.clone(), (IndexInput)indexStream.clone());
+    return new Lucene40StoredFieldsReader(fieldInfos, numTotalDocs, size, fieldsStream.clone(), indexStream.clone());
   }
   
   // Used only by clone

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40StoredFieldsWriter.java Fri Sep 21 17:21:34 2012
@@ -29,6 +29,8 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentReader;
+import org.apache.lucene.index.StorableField;
+import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -131,7 +133,7 @@ public final class Lucene40StoredFieldsW
         IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
   }
 
-  public void writeField(FieldInfo info, IndexableField field) throws IOException {
+  public void writeField(FieldInfo info, StorableField field) throws IOException {
     fieldsStream.writeVInt(info.number);
     int bits = 0;
     final BytesRef bytes;
@@ -186,7 +188,7 @@ public final class Lucene40StoredFieldsW
       } else if (number instanceof Double) {
         fieldsStream.writeLong(Double.doubleToLongBits(number.doubleValue()));
       } else {
-        assert false;
+        throw new AssertionError("Cannot get here");
       }
     }
   }
@@ -297,7 +299,7 @@ public final class Lucene40StoredFieldsW
         // on the fly?
         // NOTE: it's very important to first assign to doc then pass it to
         // fieldsWriter.addDocument; see LUCENE-1282
-        Document doc = reader.document(j);
+        StoredDocument doc = reader.document(j);
         addDocument(doc, mergeState.fieldInfos);
         docCount++;
         mergeState.checkAbort.work(300);
@@ -324,7 +326,7 @@ public final class Lucene40StoredFieldsW
       for (; docCount < maxDoc; docCount++) {
         // NOTE: it's very important to first assign to doc then pass it to
         // fieldsWriter.addDocument; see LUCENE-1282
-        Document doc = reader.document(docCount);
+        StoredDocument doc = reader.document(docCount);
         addDocument(doc, mergeState.fieldInfos);
         mergeState.checkAbort.work(300);
       }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Fri Sep 21 17:21:34 2012
@@ -403,7 +403,7 @@ public class Lucene40TermVectorsReader e
     // NOTE: tvf is pre-positioned by caller
     public TVTermsEnum() {
       this.origTVF = Lucene40TermVectorsReader.this.tvf;
-      tvf = (IndexInput) origTVF.clone();
+      tvf = origTVF.clone();
     }
 
     public boolean canReuse(IndexInput tvf) {
@@ -752,9 +752,9 @@ public class Lucene40TermVectorsReader e
     // These are null when a TermVectorsReader was created
     // on a segment that did not have term vectors saved
     if (tvx != null && tvd != null && tvf != null) {
-      cloneTvx = (IndexInput) tvx.clone();
-      cloneTvd = (IndexInput) tvd.clone();
-      cloneTvf = (IndexInput) tvf.clone();
+      cloneTvx = tvx.clone();
+      cloneTvd = tvd.clone();
+      cloneTvf = tvf.clone();
     }
     
     return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java Fri Sep 21 17:21:34 2012
@@ -315,12 +315,7 @@ public final class Lucene40TermVectorsWr
     int numDocs = 0;
     for (int i = 0; i < mergeState.readers.size(); i++) {
       final AtomicReader reader = mergeState.readers.get(i);
-      // set PayloadProcessor
-      if (mergeState.payloadProcessorProvider != null) {
-        mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
-      } else {
-        mergeState.currentReaderPayloadProcessor = null;
-      }
+
       final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
       Lucene40TermVectorsReader matchingVectorsReader = null;
       if (matchingSegmentReader != null) {
@@ -353,8 +348,8 @@ public final class Lucene40TermVectorsWr
     final int maxDoc = reader.maxDoc();
     final Bits liveDocs = reader.getLiveDocs();
     int totalNumDocs = 0;
-    if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) {
-      // We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor
+    if (matchingVectorsReader != null) {
+      // We can bulk-copy because the fieldInfos are "congruent"
       for (int docNum = 0; docNum < maxDoc;) {
         if (!liveDocs.get(docNum)) {
           // skip deleted docs
@@ -404,8 +399,8 @@ public final class Lucene40TermVectorsWr
                                       int rawDocLengths2[])
           throws IOException {
     final int maxDoc = reader.maxDoc();
-    if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) {
-      // We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor
+    if (matchingVectorsReader != null) {
+      // We can bulk-copy because the fieldInfos are "congruent"
       int docCount = 0;
       while (docCount < maxDoc) {
         int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Bytes.java Fri Sep 21 17:21:34 2012
@@ -30,6 +30,7 @@ import org.apache.lucene.index.DocValues
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -349,7 +350,7 @@ public final class Bytes {
      */
     protected final IndexInput cloneData() {
       assert datIn != null;
-      return (IndexInput) datIn.clone();
+      return datIn.clone();
     }
 
     /**
@@ -357,7 +358,7 @@ public final class Bytes {
      */
     protected final IndexInput cloneIndex() {
       assert idxIn != null;
-      return (IndexInput) idxIn.clone();
+      return idxIn.clone();
     }
 
     @Override
@@ -420,7 +421,7 @@ public final class Bytes {
     }
 
     @Override
-    public void add(int docID, IndexableField value) throws IOException {
+    public void add(int docID, StorableField value) throws IOException {
       BytesRef bytes = value.binaryValue();
       assert bytes != null;
       if (bytes.length == 0) { // default value - skip it

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/FixedStraightBytesImpl.java Fri Sep 21 17:21:34 2012
@@ -22,12 +22,12 @@ import java.io.IOException;
 import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
 import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
 import org.apache.lucene.codecs.lucene40.values.Bytes.BytesWriterBase;
+import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StraightBytesDocValuesField;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -74,7 +74,7 @@ class FixedStraightBytesImpl {
     }
     
     @Override
-    public void add(int docID, IndexableField value) throws IOException {
+    public void add(int docID, StorableField value) throws IOException {
       final BytesRef bytes = value.binaryValue();
       assert bytes != null;
       assert lastDocID < docID;
@@ -201,7 +201,7 @@ class FixedStraightBytesImpl {
     }
     
     @Override
-    protected void mergeDoc(Field scratchField, Source source, int docID, int sourceDoc) throws IOException {
+    protected void mergeDoc(StoredField scratchField, Source source, int docID, int sourceDoc) throws IOException {
       assert lastDocID < docID;
       setMergeBytes(source, sourceDoc);
       if (size == -1) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java Fri Sep 21 17:21:34 2012
@@ -24,6 +24,7 @@ import org.apache.lucene.index.DocValues
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -88,7 +89,7 @@ public class Floats {
     }
     
     @Override
-    public void add(int docID, IndexableField value) throws IOException {
+    public void add(int docID, StorableField value) throws IOException {
       template.toBytes(value.numericValue().doubleValue(), bytesRef);
       bytesSpareField.setBytesValue(bytesRef);
       super.add(docID, bytesSpareField);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/Ints.java Fri Sep 21 17:21:34 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.index.DocValues
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -112,7 +113,7 @@ public final class Ints {
     }
     
     @Override
-    public void add(int docID, IndexableField value) throws IOException {
+    public void add(int docID, StorableField value) throws IOException {
       template.toBytes(value.numericValue().longValue(), bytesRef);
       bytesSpareField.setBytesValue(bytesRef);
       super.add(docID, bytesSpareField);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/PackedIntValues.java Fri Sep 21 17:21:34 2012
@@ -26,6 +26,7 @@ import org.apache.lucene.index.DocValues
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -126,7 +127,7 @@ class PackedIntValues {
     }
     
     @Override
-    public void add(int docID, IndexableField docValue) throws IOException {
+    public void add(int docID, StorableField docValue) throws IOException {
       final long v = docValue.numericValue().longValue();
       assert lastDocId < docID;
       if (!started) {
@@ -186,7 +187,7 @@ class PackedIntValues {
       final Source source;
       IndexInput input = null;
       try {
-        input = (IndexInput) datIn.clone();
+        input = datIn.clone();
         
         if (values == null) {
           source = new PackedIntsSource(input, false);
@@ -217,7 +218,7 @@ class PackedIntValues {
 
     @Override
     public Source getDirectSource() throws IOException {
-      return values != null ? new FixedStraightBytesImpl.DirectFixedStraightSource((IndexInput) datIn.clone(), 8, Type.FIXED_INTS_64) : new PackedIntsSource((IndexInput) datIn.clone(), true);
+      return values != null ? new FixedStraightBytesImpl.DirectFixedStraightSource(datIn.clone(), 8, Type.FIXED_INTS_64) : new PackedIntsSource(datIn.clone(), true);
     }
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java Fri Sep 21 17:21:34 2012
@@ -211,7 +211,7 @@ final class VarSortedBytesImpl {
       valueCount = ordToOffsetIndex.size()-1; // the last value here is just a dummy value to get the length of the last value
       // advance this iterator to the end and clone the stream once it points to the docToOrdIndex header
       ordToOffsetIndex.get(valueCount);
-      docToOrdIndex = PackedInts.getDirectReader((IndexInput) idxIn.clone()); // read the ords in to prevent too many random disk seeks
+      docToOrdIndex = PackedInts.getDirectReader(idxIn.clone()); // read the ords in to prevent too many random disk seeks
       basePointer = datIn.getFilePointer();
       this.datIn = datIn;
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarStraightBytesImpl.java Fri Sep 21 17:21:34 2012
@@ -22,11 +22,11 @@ import java.io.IOException;
 import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
 import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
 import org.apache.lucene.codecs.lucene40.values.Bytes.BytesWriterBase;
-import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -86,7 +86,7 @@ class VarStraightBytesImpl {
     }
 
     @Override
-    public void add(int docID, IndexableField value) throws IOException {
+    public void add(int docID, StorableField value) throws IOException {
       final BytesRef bytes = value.binaryValue();
       assert bytes != null;
       assert !merge;
@@ -156,7 +156,7 @@ class VarStraightBytesImpl {
     }
     
     @Override
-    protected void mergeDoc(Field scratchField, Source source, int docID, int sourceDoc) throws IOException {
+    protected void mergeDoc(StoredField scratchField, Source source, int docID, int sourceDoc) throws IOException {
       assert merge;
       assert lastDocID < docID;
       source.getBytes(sourceDoc, bytesRef);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/package.html?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/package.html (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/package.html Fri Sep 21 17:21:34 2012
@@ -35,6 +35,9 @@ Codecs API: API for customization of the
   <li>Live documents - see {@link org.apache.lucene.codecs.LiveDocsFormat}</li>
 </ul>
 </p>
+ 
+  For some concrete implementations beyond Lucene's official index format, see
+  the <a href="{@docRoot}/../codecs/overview-summary.html">Codecs module</a>.
 
 <p>
   Codecs are identified by name through the Java Service Provider Interface.  To create your own codec, extend

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Fri Sep 21 17:21:34 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.perfiel
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -34,7 +35,6 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.UnmodifiableIterator;
 
 /**
  * Enables per field format support.
@@ -63,7 +63,7 @@ public abstract class PerFieldPostingsFo
   }
 
   @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state)
+  public final FieldsConsumer fieldsConsumer(SegmentWriteState state)
       throws IOException {
     return new FieldsWriter(state);
   }
@@ -199,7 +199,7 @@ public abstract class PerFieldPostingsFo
 
     @Override
     public Iterator<String> iterator() {
-      return new UnmodifiableIterator<String>(fields.keySet().iterator());
+      return Collections.unmodifiableSet(fields.keySet()).iterator();
     }
 
     @Override
@@ -220,13 +220,16 @@ public abstract class PerFieldPostingsFo
   }
 
   @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state)
+  public final FieldsProducer fieldsProducer(SegmentReadState state)
       throws IOException {
     return new FieldsReader(state);
   }
 
-  // NOTE: only called during writing; for reading we read
-  // all we need from the index (ie we save the field ->
-  // format mapping)
+  /** 
+   * Returns the postings format that should be used for writing 
+   * new segments of <code>field</code>.
+   * <p>
+   * The field to format mapping is written to the index, so
+   * this method is only invoked when writing, not when reading. */
   public abstract PostingsFormat getPostingsFormatForField(String field);
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/ByteDocValuesField.java Fri Sep 21 17:21:34 2012
@@ -21,11 +21,10 @@ import org.apache.lucene.index.DocValues
 
 /**
  * <p>
- * This class provides a {@link Field} that enables storing
- * of a per-document byte value for scoring, sorting or value retrieval. Here's an
- * example usage:
+ * Field that stores a per-document <code>byte</code> value for scoring, 
+ * sorting or value retrieval. Here's an example usage:
  * 
- * <pre>
+ * <pre class="prettyprint">
  *   document.add(new ByteDocValuesField(name, (byte) 22));
  * </pre>
  * 
@@ -33,17 +32,26 @@ import org.apache.lucene.index.DocValues
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues for further information
+ * @see DocValues
  * */
 
-public class ByteDocValuesField extends Field {
+public class ByteDocValuesField extends StoredField {
 
+  /**
+   * Type for 8-bit byte DocValues.
+   */
   public static final FieldType TYPE = new FieldType();
   static {
     TYPE.setDocValueType(DocValues.Type.FIXED_INTS_8);
     TYPE.freeze();
   }
 
+  /** 
+   * Creates a new DocValues field with the specified 8-bit byte value 
+   * @param name field name
+   * @param value 8-bit byte value
+   * @throws IllegalArgumentException if the field name is null.
+   */
   public ByteDocValuesField(String name, byte value) {
     super(name, TYPE);
     fieldsData = Byte.valueOf(value);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/CompressionTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/CompressionTools.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/CompressionTools.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/CompressionTools.java Fri Sep 21 17:21:34 2012
@@ -92,10 +92,14 @@ public class CompressionTools {
     return compress(result.bytes, 0, result.length, compressionLevel);
   }
 
+  /** Decompress the byte array previously returned by
+   *  compress (referenced by the provided BytesRef) */
   public static byte[] decompress(BytesRef bytes) throws DataFormatException {
     return decompress(bytes.bytes, bytes.offset, bytes.length);
   }
 
+  /** Decompress the byte array previously returned by
+   *  compress */
   public static byte[] decompress(byte[] value) throws DataFormatException {
     return decompress(value, 0, value.length);
   }
@@ -130,6 +134,8 @@ public class CompressionTools {
     return decompressString(value, 0, value.length);
   }
 
+  /** Decompress the byte array previously returned by
+   *  compressString back into a String */
   public static String decompressString(byte[] value, int offset, int length) throws DataFormatException {
     final byte[] bytes = decompress(value, offset, length);
     CharsRef result = new CharsRef(bytes.length);
@@ -137,6 +143,8 @@ public class CompressionTools {
     return new String(result.chars, 0, result.length);
   }
 
+  /** Decompress the byte array (referenced by the provided BytesRef) 
+   *  previously returned by compressString back into a String */
   public static String decompressString(BytesRef bytes) throws DataFormatException {
     return decompressString(bytes.bytes, bytes.offset, bytes.length);
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DateTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DateTools.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DateTools.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DateTools.java Fri Sep 21 17:21:34 2012
@@ -185,7 +185,20 @@ public class DateTools {
   /** Specifies the time granularity. */
   public static enum Resolution {
     
-    YEAR(4), MONTH(6), DAY(8), HOUR(10), MINUTE(12), SECOND(14), MILLISECOND(17);
+    /** Limit a date's resolution to year granularity. */
+    YEAR(4), 
+    /** Limit a date's resolution to month granularity. */
+    MONTH(6), 
+    /** Limit a date's resolution to day granularity. */
+    DAY(8), 
+    /** Limit a date's resolution to hour granularity. */
+    HOUR(10), 
+    /** Limit a date's resolution to minute granularity. */
+    MINUTE(12), 
+    /** Limit a date's resolution to second granularity. */
+    SECOND(14), 
+    /** Limit a date's resolution to millisecond granularity. */
+    MILLISECOND(17);
 
     final int formatLen;
     final SimpleDateFormat format;//should be cloned before use, since it's not threadsafe

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DerefBytesDocValuesField.java Fri Sep 21 17:21:34 2012
@@ -22,15 +22,15 @@ import org.apache.lucene.util.BytesRef;
 
 /**
  * <p>
- * This class provides a {@link Field} that enables storing
- * of a per-document {@link BytesRef} value.  The values are
+ * Field that stores
+ * a per-document {@link BytesRef} value.  The values are
  * stored indirectly, such that many documents sharing the
  * same value all point to a single copy of the value, which
  * is a good fit when the fields share values.  If values
  * are (mostly) unique it's better to use {@link
  * StraightBytesDocValuesField}.  Here's an example usage: 
  * 
- * <pre>
+ * <pre class="prettyprint">
  *   document.add(new DerefBytesDocValuesField(name, new BytesRef("hello")));
  * </pre>
  * 
@@ -38,29 +38,55 @@ import org.apache.lucene.util.BytesRef;
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues for further information
+ * @see DocValues
  * */
 
-public class DerefBytesDocValuesField extends Field {
+public class DerefBytesDocValuesField extends StoredField {
 
   // TODO: ideally indexer figures out var vs fixed on its own!?
+  /**
+   * Type for indirect bytes DocValues: all with the same length
+   */
   public static final FieldType TYPE_FIXED_LEN = new FieldType();
   static {
     TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_DEREF);
     TYPE_FIXED_LEN.freeze();
   }
 
+  /**
+   * Type for indirect bytes DocValues: can have variable lengths
+   */
   public static final FieldType TYPE_VAR_LEN = new FieldType();
   static {
     TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_DEREF);
     TYPE_VAR_LEN.freeze();
   }
 
+  /**
+   * Create a new variable-length indirect DocValues field.
+   * <p>
+   * This calls 
+   * {@link DerefBytesDocValuesField#DerefBytesDocValuesField(String, BytesRef, boolean)
+   *  DerefBytesDocValuesField(name, bytes, false}, meaning by default
+   * it allows for values of different lengths. If your values are all 
+   * the same length, use that constructor instead.
+   * @param name field name
+   * @param bytes binary content
+   * @throws IllegalArgumentException if the field name is null
+   */
   public DerefBytesDocValuesField(String name, BytesRef bytes) {
     super(name, TYPE_VAR_LEN);
     fieldsData = bytes;
   }
 
+  /**
+   * Create a new fixed or variable length indirect DocValues field.
+   * <p>
+   * @param name field name
+   * @param bytes binary content
+   * @param isFixedLength true if all values have the same length.
+   * @throws IllegalArgumentException if the field name is null
+   */
   public DerefBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
     super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
     fieldsData = bytes;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/Document.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/Document.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/Document.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/Document.java Fri Sep 21 17:21:34 2012
@@ -19,11 +19,15 @@ package org.apache.lucene.document;
 
 import java.util.*;
 
+import org.apache.lucene.index.IndexDocument;
 import org.apache.lucene.index.IndexReader;  // for javadoc
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.StorableField;
+import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.search.IndexSearcher;  // for javadoc
 import org.apache.lucene.search.ScoreDoc; // for javadoc
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FilterIterator;
 
 /** Documents are the unit of indexing and search.
  *
@@ -38,18 +42,36 @@ import org.apache.lucene.util.BytesRef;
  * ScoreDoc#doc} or {@link IndexReader#document(int)}.
  */
 
-public final class Document implements Iterable<IndexableField> {
+public final class Document implements IndexDocument {
 
-  private final List<IndexableField> fields = new ArrayList<IndexableField>();
+  private final List<Field> fields = new ArrayList<Field>();
 
   /** Constructs a new document with no fields. */
   public Document() {}
+  
 
-  @Override
-  public Iterator<IndexableField> iterator() {
-    return fields.iterator();
-  }
+  /**
+  * Creates a Document from StoredDocument so it that can be used e.g. for another
+  * round of indexing.
+  *
+  */
+  public Document(StoredDocument storedDoc) {
+    for (StorableField field : storedDoc.getFields()) {
+      Field newField = new Field(field.name(), (FieldType) field.fieldType());
+     
+      newField.fieldsData = field.stringValue();
+      if (newField.fieldsData == null) 
+        newField.fieldsData = field.numericValue();
+      if (newField.fieldsData == null) 
+        newField.fieldsData = field.binaryValue();
+      if (newField.fieldsData == null) 
+        newField.fieldsData = field.readerValue();
+     
+      add(newField);
+    }
+ }
 
+  
   /**
    * <p>Adds a field to a document.  Several fields may be added with
    * the same name.  In this case, if the fields are indexed, their text is
@@ -60,7 +82,7 @@ public final class Document implements I
    * a document has to be deleted from an index and a new changed version of that
    * document has to be added.</p>
    */
-  public final void add(IndexableField field) {
+  public final void add(Field field) {
     fields.add(field);
   }
   
@@ -75,9 +97,9 @@ public final class Document implements I
    * document has to be added.</p>
    */
   public final void removeField(String name) {
-    Iterator<IndexableField> it = fields.iterator();
+    Iterator<Field> it = fields.iterator();
     while (it.hasNext()) {
-      IndexableField field = it.next();
+      Field field = it.next();
       if (field.name().equals(name)) {
         it.remove();
         return;
@@ -95,9 +117,9 @@ public final class Document implements I
    * document has to be added.</p>
    */
   public final void removeFields(String name) {
-    Iterator<IndexableField> it = fields.iterator();
+    Iterator<Field> it = fields.iterator();
     while (it.hasNext()) {
-      IndexableField field = it.next();
+      Field field = it.next();
       if (field.name().equals(name)) {
         it.remove();
       }
@@ -112,11 +134,14 @@ public final class Document implements I
   * returns null.
   *
   * @param name the name of the field
-  * @return a <code>byte[][]</code> of binary field values
+  * @return a <code>BytesRef[]</code> of binary field values
   */
   public final BytesRef[] getBinaryValues(String name) {
     final List<BytesRef> result = new ArrayList<BytesRef>();
-    for (IndexableField field : fields) {
+    Iterator<Field> it = storedFieldsIterator();
+    
+    while (it.hasNext()) {
+      StorableField field = it.next();
       if (field.name().equals(name)) {
         final BytesRef bytes = field.binaryValue();
         if (bytes != null) {
@@ -135,10 +160,13 @@ public final class Document implements I
   * There may be non-binary fields with the same name.
   *
   * @param name the name of the field.
-  * @return a <code>byte[]</code> containing the binary field value or <code>null</code>
+  * @return a <code>BytesRef</code> containing the binary field value or <code>null</code>
   */
   public final BytesRef getBinaryValue(String name) {
-    for (IndexableField field : fields) {
+    Iterator<Field> it = storedFieldsIterator();
+    
+    while (it.hasNext()) {
+      StorableField field = it.next();
       if (field.name().equals(name)) {
         final BytesRef bytes = field.binaryValue();
         if (bytes != null) {
@@ -153,8 +181,8 @@ public final class Document implements I
    * null.  If multiple fields exists with this name, this method returns the
    * first value added.
    */
-  public final IndexableField getField(String name) {
-    for (IndexableField field : fields) {
+  public final Field getField(String name) {
+    for (Field field : fields) {
       if (field.name().equals(name)) {
         return field;
       }
@@ -168,17 +196,17 @@ public final class Document implements I
    * matching fields.  It never returns null.
    *
    * @param name the name of the field
-   * @return a <code>Fieldable[]</code> array
+   * @return a <code>Field[]</code> array
    */
-  public IndexableField[] getFields(String name) {
-    List<IndexableField> result = new ArrayList<IndexableField>();
-    for (IndexableField field : fields) {
+  public Field[] getFields(String name) {
+    List<Field> result = new ArrayList<Field>();
+    for (Field field : fields) {
       if (field.name().equals(name)) {
         result.add(field);
       }
     }
 
-    return result.toArray(new IndexableField[result.size()]);
+    return result.toArray(new Field[result.size()]);
   }
   
   /** Returns a List of all the fields in a document.
@@ -186,9 +214,11 @@ public final class Document implements I
    * <i>not</i> available in documents retrieved from the
    * index, e.g. {@link IndexSearcher#doc(int)} or {@link
    * IndexReader#document(int)}.
+   * 
+   * @return an immutable <code>List&lt;Field&gt;</code> 
    */
-  public final List<IndexableField> getFields() {
-    return fields;
+  public final List<Field> getFields() {
+    return Collections.unmodifiableList(fields);
   }
   
    private final static String[] NO_STRINGS = new String[0];
@@ -205,7 +235,10 @@ public final class Document implements I
    */
   public final String[] getValues(String name) {
     List<String> result = new ArrayList<String>();
-    for (IndexableField field : fields) {
+    Iterator<Field> it = storedFieldsIterator();
+    
+    while (it.hasNext()) {
+      StorableField field = it.next();
       if (field.name().equals(name) && field.stringValue() != null) {
         result.add(field.stringValue());
       }
@@ -227,7 +260,10 @@ public final class Document implements I
    * the actual numeric field instance back, use {@link #getField}.
    */
   public final String get(String name) {
-    for (IndexableField field : fields) {
+    Iterator<Field> it = storedFieldsIterator();
+    
+    while (it.hasNext()) {
+      StorableField field = it.next();
       if (field.name().equals(name) && field.stringValue() != null) {
         return field.stringValue();
       }
@@ -249,4 +285,50 @@ public final class Document implements I
     buffer.append(">");
     return buffer.toString();
   }
+
+  /** Obtains all indexed fields in document */
+  @Override
+  public Iterable<? extends IndexableField> indexableFields() {
+    return new Iterable<Field>() {
+      @Override
+      public Iterator<Field> iterator() {
+        return Document.this.indexedFieldsIterator();
+      }
+    };
+  }
+
+
+  /** Obtains all stored fields in document. */
+  @Override
+  public Iterable<? extends StorableField> storableFields() {
+    return new Iterable<Field>() {
+      @Override
+      public Iterator<Field> iterator() {
+        return Document.this.storedFieldsIterator();
+      }
+    };
+  }
+
+  private Iterator<Field> storedFieldsIterator() {
+    return new FilterIterator<Field>(fields.iterator()) {
+      @Override
+      protected boolean predicateFunction(Field field) {
+        return field.type.stored() || field.type.docValueType() != null;
+      }
+    };
+  }
+  
+  private Iterator<Field> indexedFieldsIterator() {
+    return new FilterIterator<Field>(fields.iterator()) {
+      @Override
+      protected boolean predicateFunction(Field field) {
+        return field.type.indexed();
+      }
+    };
+  }
+
+  /** Removes all the fields from document. */
+  public void clear() {
+    fields.clear();
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java Fri Sep 21 17:21:34 2012
@@ -23,18 +23,20 @@ import java.util.HashSet;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.index.StoredFieldVisitor;
 
 /** A {@link StoredFieldVisitor} that creates a {@link
  *  Document} containing all stored fields, or only specific
- *  requested fields provided to {@link #DocumentStoredFieldVisitor(Set)}
+ *  requested fields provided to {@link #DocumentStoredFieldVisitor(Set)}.
+ *  <p>
  *  This is used by {@link IndexReader#document(int)} to load a
  *  document.
  *
  * @lucene.experimental */
 
 public class DocumentStoredFieldVisitor extends StoredFieldVisitor {
-  private final Document doc = new Document();
+  private final StoredDocument doc = new StoredDocument();
   private final Set<String> fieldsToAdd;
 
   /** Load only fields named in the provided <code>Set&lt;String&gt;</code>. */
@@ -67,7 +69,7 @@ public class DocumentStoredFieldVisitor 
     ft.setIndexed(fieldInfo.isIndexed());
     ft.setOmitNorms(fieldInfo.omitsNorms());
     ft.setIndexOptions(fieldInfo.getIndexOptions());
-    doc.add(new Field(fieldInfo.name, value, ft));
+    doc.add(new StoredField(fieldInfo.name, value, ft));
   }
 
   @Override
@@ -95,7 +97,14 @@ public class DocumentStoredFieldVisitor 
     return fieldsToAdd == null || fieldsToAdd.contains(fieldInfo.name) ? Status.YES : Status.NO;
   }
 
-  public Document getDocument() {
+  /**
+   * Retrieve the visited document.
+   * @return {@link StoredDocument} populated with stored fields. Note that only
+   *         the stored information in the field instances is valid,
+   *         data such as indexing options, term vector options,
+   *         etc is not set.
+   */
+  public StoredDocument getDocument() {
     return doc;
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java Fri Sep 21 17:21:34 2012
@@ -21,11 +21,10 @@ import org.apache.lucene.index.DocValues
 
 /**
  * <p>
- * This class provides a {@link Field} that enables storing
- * of a per-document double value for scoring, sorting or value retrieval. Here's an
- * example usage:
+ * Field that stores a per-document <code>double</code> value for scoring, 
+ * sorting or value retrieval. Here's an example usage:
  * 
- * <pre>
+ * <pre class="prettyprint">
  *   document.add(new DoubleDocValuesField(name, 22.0));
  * </pre>
  * 
@@ -33,17 +32,26 @@ import org.apache.lucene.index.DocValues
  * If you also need to store the value, you should add a
  * separate {@link StoredField} instance.
  * 
- * @see DocValues for further information
+ * @see DocValues
  * */
 
-public class DoubleDocValuesField extends Field {
+public class DoubleDocValuesField extends StoredField {
 
+  /**
+   * Type for 64-bit double DocValues.
+   */
   public static final FieldType TYPE = new FieldType();
   static {
     TYPE.setDocValueType(DocValues.Type.FLOAT_64);
     TYPE.freeze();
   }
 
+  /** 
+   * Creates a new DocValues field with the specified 64-bit double value 
+   * @param name field name
+   * @param value 64-bit double value
+   * @throws IllegalArgumentException if the field name is null
+   */
   public DoubleDocValuesField(String name, double value) {
     super(name, TYPE);
     fieldsData = Double.valueOf(value);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DoubleField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DoubleField.java?rev=1388574&r1=1388573&r2=1388574&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DoubleField.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DoubleField.java Fri Sep 21 17:21:34 2012
@@ -26,17 +26,17 @@ import org.apache.lucene.util.NumericUti
 
 /**
  * <p>
- * This class provides a {@link Field} that enables indexing of double values
+ * Field that indexes <code>double</code> values
  * for efficient range filtering and sorting. Here's an example usage:
  * 
- * <pre>
+ * <pre class="prettyprint">
  * document.add(new DoubleField(name, 6.0, Field.Store.NO));
  * </pre>
  * 
  * For optimal performance, re-use the <code>DoubleField</code> and
  * {@link Document} instance for more than one document:
  * 
- * <pre>
+ * <pre class="prettyprint">
  *  DoubleField field = new DoubleField(name, 0.0, Field.Store.NO);
  *  Document document = new Document();
  *  document.add(field);
@@ -114,6 +114,10 @@ import org.apache.lucene.util.NumericUti
 
 public final class DoubleField extends Field {
   
+  /** 
+   * Type for a DoubleField that is not stored:
+   * normalization factors, frequencies, and positions are omitted.
+   */
   public static final FieldType TYPE_NOT_STORED = new FieldType();
   static {
     TYPE_NOT_STORED.setIndexed(true);
@@ -124,6 +128,10 @@ public final class DoubleField extends F
     TYPE_NOT_STORED.freeze();
   }
 
+  /** 
+   * Type for a stored DoubleField:
+   * normalization factors, frequencies, and positions are omitted.
+   */
   public static final FieldType TYPE_STORED = new FieldType();
   static {
     TYPE_STORED.setIndexed(true);
@@ -137,14 +145,26 @@ public final class DoubleField extends F
 
   /** Creates a stored or un-stored DoubleField with the provided value
    *  and default <code>precisionStep</code> {@link
-   *  NumericUtils#PRECISION_STEP_DEFAULT} (4). */
+   *  NumericUtils#PRECISION_STEP_DEFAULT} (4). 
+   *  @param name field name
+   *  @param value 64-bit double value
+   *  @param stored Store.YES if the content should also be stored
+   *  @throws IllegalArgumentException if the field name is null. 
+   */
   public DoubleField(String name, double value, Store stored) {
     super(name, stored == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
     fieldsData = Double.valueOf(value);
   }
   
   /** Expert: allows you to customize the {@link
-   *  FieldType}. */
+   *  FieldType}. 
+   *  @param name field name
+   *  @param value 64-bit double value
+   *  @param type customized field type: must have {@link FieldType#numericType()}
+   *         of {@link FieldType.NumericType#DOUBLE}.
+   *  @throws IllegalArgumentException if the field name or type is null, or
+   *          if the field type does not have a DOUBLE numericType()
+   */
   public DoubleField(String name, double value, FieldType type) {
     super(name, type);
     if (type.numericType() != FieldType.NumericType.DOUBLE) {



Mime
View raw message