You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/08/25 12:06:16 UTC

svn commit: r1377246 [3/7] - in /lucene/dev/branches/lucene3312: ./ dev-tools/ dev-tools/idea/lucene/analysis/morfologik/ dev-tools/idea/lucene/analysis/phonetic/ dev-tools/maven/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ luce...

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/package.html?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/package.html (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/package.html Sat Aug 25 10:06:07 2012
@@ -21,6 +21,241 @@
 </head>
 <body>
 Code to maintain and access indices.
-<!-- TODO: add a BASIC overview here, including code examples of using postings apis -->
+<!-- TODO: add IndexWriter, IndexWriterConfig, DocValues, etc etc -->
+<h2>Table Of Contents</h2>
+<p>
+    <ol>
+        <li><a href="#postings">Postings APIs</a>
+            <ul>
+                <li><a href="#fields">Fields</a></li>
+                <li><a href="#terms">Terms</a></li>
+                <li><a href="#documents">Documents</a></li>
+                <li><a href="#positions">Positions</a></li>
+            </ul>
+        </li>
+        <li><a href="#stats">Index Statistics</a>
+            <ul>
+                <li><a href="#termstats">Term-level</a></li>
+                <li><a href="#fieldstats">Field-level</a></li>
+                <li><a href="#segmentstats">Segment-level</a></li>
+                <li><a href="#documentstats">Document-level</a></li>
+            </ul>
+        </li>
+    </ol>
+</p>
+<a name="postings"></a>
+<h2>Postings APIs</h2>
+<a name="fields"></a>
+<h4>
+    Fields
+</h4>
+<p>
+{@link org.apache.lucene.index.Fields} is the initial entry point into the 
+postings APIs, this can be obtained in several ways:
+<pre class="prettyprint">
+// access indexed fields for an index segment
+Fields fields = reader.fields();
+// access term vector fields for a specified document
+Fields fields = reader.getTermVectors(docid);
+</pre>
+Fields implements Java's Iterable interface, so its easy to enumerate the
+list of fields:
+<pre class="prettyprint">
+// enumerate list of fields
+for (String field : fields) {
+  // access the terms for this field
+  Terms terms = fields.terms(field);
+}
+</pre>
+</p>
+<a name="terms"></a>
+<h4>
+    Terms
+</h4>
+<p>
+{@link org.apache.lucene.index.Terms} represents the collection of terms
+within a field, exposes some metadata and <a href="#fieldstats">statistics</a>,
+and an API for enumeration.
+<pre class="prettyprint">
+// metadata about the field
+System.out.println("positions? " + terms.hasPositions());
+System.out.println("offsets? " + terms.hasOffsets());
+System.out.println("payloads? " + terms.hasPayloads());
+// iterate through terms
+TermsEnum termsEnum = terms.iterator(null);
+BytesRef term = null;
+while ((term = termsEnum.next()) != null) {
+  doSomethingWith(termsEnum.term());
+}
+</pre>
+{@link org.apache.lucene.index.TermsEnum} provides an iterator over the list
+of terms within a field, some <a href="#termstats">statistics</a> about the term,
+and methods to access the term's <a href="#documents">documents</a> and
+<a href="#positions">positions</a>.
+<pre class="prettyprint">
+// seek to a specific term
+boolean found = termsEnum.seekExact(new BytesRef("foobar"), true);
+if (found) {
+  // get the document frequency
+  System.out.println(termsEnum.docFreq());
+  // enumerate through documents
+  DocsEnum docs = termsEnum.docs(null, null);
+  // enumerate through documents and positions
+  DocsAndPositionsEnum docsAndPositions = termsEnum.docsAndPositions(null, null);
+}
+</pre>
+</p>
+<a name="documents"></a>
+<h4>
+    Documents
+</h4>
+<p>
+{@link org.apache.lucene.index.DocsEnum} is an extension of 
+{@link org.apache.lucene.search.DocIdSetIterator}that iterates over the list of
+documents for a term, along with the term frequency within that document.
+<pre class="prettyprint">
+int docid;
+while ((docid = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+  System.out.println(docid);
+  System.out.println(docsEnum.freq());
+}
+</pre>
+</p>
+<a name="positions"></a>
+<h4>
+    Positions
+</h4>
+<p>
+{@link org.apache.lucene.index.DocsAndPositionsEnum} is an extension of 
+{@link org.apache.lucene.index.DocsEnum} that additionally allows iteration
+of the positions a term occurred within the document, and any additional
+per-position information (offsets and payload)
+<pre class="prettyprint">
+int docid;
+while ((docid = docsAndPositionsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+  System.out.println(docid);
+  int freq = docsAndPositionsEnum.freq();
+  for (int i = 0; i < freq; i++) {
+     System.out.println(docsAndPositionsEnum.nextPosition());
+     System.out.println(docsAndPositionsEnum.startOffset());
+     System.out.println(docsAndPositionsEnum.endOffset());
+     System.out.println(docsAndPositionsEnum.getPayload());
+  }
+}
+</pre>
+</p>
+<a name="stats"></a>
+<h2>Index Statistics</h2>
+<a name="termstats"></a>
+<h4>
+    Term statistics
+</h4>
+<p>
+    <ul>
+       <li>{@link org.apache.lucene.index.TermsEnum#docFreq}: Returns the number of 
+           documents that contain at least one occurrence of the term. This statistic 
+           is always available for an indexed term. Note that it will also count 
+           deleted documents, when segments are merged the statistic is updated as 
+           those deleted documents are merged away.
+       <li>{@link org.apache.lucene.index.TermsEnum#totalTermFreq}: Returns the number 
+           of occurrences of this term across all documents. Note that this statistic 
+           is unavailable (returns <code>-1</code>) if term frequencies were omitted 
+           from the index 
+           ({@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_ONLY DOCS_ONLY}) 
+           for the field. Like docFreq(), it will also count occurrences that appear in 
+           deleted documents.
+    </ul>
+</p>
+<a name="fieldstats"></a>
+<h4>
+    Field statistics
+</h4>
+<p>
+    <ul>
+       <li>{@link org.apache.lucene.index.Terms#size}: Returns the number of 
+           unique terms in the field. This statistic may be unavailable 
+           (returns <code>-1</code>) for some Terms implementations such as
+           {@link org.apache.lucene.index.MultiTerms}, where it cannot be efficiently
+           computed.  Note that this count also includes terms that appear only
+           in deleted documents: when segments are merged such terms are also merged
+           away and the statistic is then updated.
+       <li>{@link org.apache.lucene.index.Terms#getDocCount}: Returns the number of
+           documents that contain at least one occurrence of any term for this field.
+           This can be thought of as a Field-level docFreq(). Like docFreq() it will
+           also count deleted documents.
+       <li>{@link org.apache.lucene.index.Terms#getSumDocFreq}: Returns the number of
+           postings (term-document mappings in the inverted index) for the field. This
+           can be thought of as the sum of {@link org.apache.lucene.index.TermsEnum#docFreq}
+           across all terms in the field, and like docFreq() it will also count postings
+           that appear in deleted documents.
+       <li>{@link org.apache.lucene.index.Terms#getSumTotalTermFreq}: Returns the number
+           of tokens for the field. This can be thought of as the sum of 
+           {@link org.apache.lucene.index.TermsEnum#totalTermFreq} across all terms in the
+           field, and like totalTermFreq() it will also count occurrences that appear in
+           deleted documents, and will be unavailable (returns <code>-1</code>) if term 
+           frequencies were omitted from the index 
+           ({@link org.apache.lucene.index.FieldInfo.IndexOptions#DOCS_ONLY DOCS_ONLY}) 
+           for the field.
+    </ul>
+</p>
+<a name="segmentstats"></a>
+<h4>
+    Segment statistics
+</h4>
+<p>
+    <ul>
+       <li>{@link org.apache.lucene.index.IndexReader#maxDoc}: Returns the number of 
+           documents (including deleted documents) in the index. 
+       <li>{@link org.apache.lucene.index.IndexReader#numDocs}: Returns the number 
+           of live documents (excluding deleted documents) in the index.
+       <li>{@link org.apache.lucene.index.IndexReader#numDeletedDocs}: Returns the
+           number of deleted documents in the index.
+       <li>{@link org.apache.lucene.index.Fields#size}: Returns the number of indexed
+           fields.
+    </ul>
+</p>
+<a name="documentstats"></a>
+<h4>
+    Document statistics
+</h4>
+<p>
+Document statistics are available during the indexing process for an indexed field: typically
+a {@link org.apache.lucene.search.similarities.Similarity} implementation will store some
+of these values (possibly in a lossy way), into the normalization value for the document in
+its {@link org.apache.lucene.search.similarities.Similarity#computeNorm} method.
+</p>
+<p>
+    <ul>
+       <li>{@link org.apache.lucene.index.FieldInvertState#getLength}: Returns the number of 
+           tokens for this field in the document. Note that this is just the number
+           of times that {@link org.apache.lucene.analysis.TokenStream#incrementToken} returned
+           true, and is unrelated to the values in 
+           {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute}.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getNumOverlap}: Returns the number
+           of tokens for this field in the document that had a position increment of zero. This
+           can be used to compute a document length that discounts artificial tokens
+           such as synonyms.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getPosition}: Returns the accumulated
+           position value for this field in the document: computed from the values of
+           {@link org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute} and including
+           {@link org.apache.lucene.analysis.Analyzer#getPositionIncrementGap}s across multivalued
+           fields.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getOffset}: Returns the total
+           character offset value for this field in the document: computed from the values of
+           {@link org.apache.lucene.analysis.tokenattributes.OffsetAttribute} returned by 
+           {@link org.apache.lucene.analysis.TokenStream#end}, and including
+           {@link org.apache.lucene.analysis.Analyzer#getOffsetGap}s across multivalued
+           fields.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getUniqueTermCount}: Returns the number
+           of unique terms encountered for this field in the document.
+       <li>{@link org.apache.lucene.index.FieldInvertState#getMaxTermFrequency}: Returns the maximum
+           frequency across all unique terms encountered for this field in the document. 
+    </ul>
+</p>
+<p>
+Additional user-supplied statistics can be added to the document as DocValues fields and
+accessed via {@link org.apache.lucene.index.AtomicReader#docValues}.
+</p>
+<p>
 </body>
 </html>

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Sat Aug 25 10:06:07 2012
@@ -23,7 +23,6 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 
 /* Description from Doug Cutting (excerpted from
@@ -115,7 +114,7 @@ final class BooleanScorer extends Scorer
   // Therefore the only methods that are implemented are score() and doc().
   private static final class BucketScorer extends Scorer {
 
-    float score;
+    double score;
     int doc = NO_MORE_DOCS;
     int freq;
     
@@ -134,13 +133,13 @@ final class BooleanScorer extends Scorer
     public int nextDoc() { return NO_MORE_DOCS; }
     
     @Override
-    public float score() { return score; }
+    public float score() { return (float)score; }
     
   }
 
   static final class Bucket {
     int doc = -1;            // tells if bucket is valid
-    float score;             // incremental score
+    double score;             // incremental score
     // TODO: break out bool anyProhibited, int
     // numRequiredMatched; then we can remove 32 limit on
     // required clauses

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java Sat Aug 25 10:06:07 2012
@@ -129,6 +129,7 @@ class ConjunctionScorer extends Scorer {
   
   @Override
   public float score() throws IOException {
+    // TODO: sum into a double and cast to float if we ever send required clauses to BS1
     float sum = 0.0f;
     for (int i = 0; i < scorers.length; i++) {
       sum += scorers[i].score();

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java Sat Aug 25 10:06:07 2012
@@ -90,6 +90,7 @@ class ConjunctionTermScorer extends Scor
 
   @Override
   public float score() throws IOException {
+    // TODO: sum into a double and cast to float if we ever send required clauses to BS1
     float sum = 0.0f;
     for (DocsAndFreqs docs : docsAndFreqs) {
       sum += docs.scorer.score();

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/DocIdSetIterator.java Sat Aug 25 10:06:07 2012
@@ -60,13 +60,17 @@ public abstract class DocIdSetIterator {
    */
   public abstract int nextDoc() throws IOException;
 
-  /**
-   * Advances to the first beyond (see NOTE below) the current whose document
-   * number is greater than or equal to <i>target</i>. Returns the current
-   * document number or {@link #NO_MORE_DOCS} if there are no more docs in the
-   * set.
+ /**
+   * Advances to the first beyond the current whose document number is greater 
+   * than or equal to <i>target</i>, and returns the document number itself. 
+   * Exhausts the iterator and returns {@link #NO_MORE_DOCS} if <i>target</i> 
+   * is greater than the highest document number in the set.
    * <p>
-   * Behaves as if written:
+   * The behavior of this method is <b>undefined</b> when called with
+   * <code> target &le; current</code>, or after the iterator has exhausted.
+   * Both cases may result in unpredicted behavior.
+   * <p>
+   * When <code> target &gt; current</code> it behaves as if written:
    * 
    * <pre>
    * int advance(int target) {
@@ -79,18 +83,12 @@ public abstract class DocIdSetIterator {
    * 
    * Some implementations are considerably more efficient than that.
    * <p>
-   * <b>NOTE:</b> when <code> target &le; current</code> implementations may opt
-   * not to advance beyond their current {@link #docID()}.
-   * <p>
    * <b>NOTE:</b> this method may be called with {@link #NO_MORE_DOCS} for
    * efficiency by some Scorers. If your implementation cannot efficiently
    * determine that it should exhaust, it is recommended that you check for that
    * value in each call to this method.
    * <p>
-   * <b>NOTE:</b> after the iterator has exhausted you should not call this
-   * method, as it may result in unpredicted behavior.
-   * <p>
-   * 
+   *
    * @since 2.9
    */
   public abstract int advance(int target) throws IOException;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Sat Aug 25 10:06:07 2012
@@ -1110,14 +1110,8 @@ class FieldCacheImpl implements FieldCac
         // Try for coarse estimate for number of bits; this
         // should be an underestimate most of the time, which
         // is fine -- GrowableWriter will reallocate as needed
-        long numUniqueTerms = 0;
-        try {
-          numUniqueTerms = terms.size();
-        } catch (UnsupportedOperationException uoe) {
-          numUniqueTerms = -1;
-        }
-        if (numUniqueTerms != -1) {
-
+        long numUniqueTerms = terms.size();
+        if (numUniqueTerms != -1L) {
           if (numUniqueTerms > termCountHardLimit) {
             // app is misusing the API (there is more than
             // one term per doc); in this case we make best
@@ -1248,13 +1242,8 @@ class FieldCacheImpl implements FieldCac
         // Try for coarse estimate for number of bits; this
         // should be an underestimate most of the time, which
         // is fine -- GrowableWriter will reallocate as needed
-        long numUniqueTerms = 0;
-        try {
-          numUniqueTerms = terms.size();
-        } catch (UnsupportedOperationException uoe) {
-          numUniqueTerms = -1;
-        }
-        if (numUniqueTerms != -1) {
+        long numUniqueTerms = terms.size();
+        if (numUniqueTerms != -1L) {
           if (numUniqueTerms > termCountHardLimit) {
             numUniqueTerms = termCountHardLimit;
           }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Sat Aug 25 10:06:07 2012
@@ -83,9 +83,7 @@ public class FilteredQuery extends Query
       
       @Override
       public boolean scoresDocsOutOfOrder() {
-        // TODO: Support out-of-order scoring!
-        // For now we return false here, as we always get the scorer in order
-        return false;
+        return true;
       }
 
       @Override
@@ -148,9 +146,7 @@ public class FilteredQuery extends Query
 
         if (useRandomAccess) {
           // if we are using random access, we return the inner scorer, just with other acceptDocs
-          // TODO, replace this by when BooleanWeight is fixed to be consistent with its scorer implementations:
-          // return weight.scorer(context, scoreDocsInOrder, topScorer, filterAcceptDocs);
-          return weight.scorer(context, true, topScorer, filterAcceptDocs);
+          return weight.scorer(context, scoreDocsInOrder, topScorer, filterAcceptDocs);
         } else {
           assert firstFilterDoc > -1;
           // we are gonna advance() this scorer, so we set inorder=true/toplevel=false

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/Query.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/Query.java Sat Aug 25 10:06:07 2012
@@ -28,17 +28,21 @@ import org.apache.lucene.index.Term;
     <p>Instantiable subclasses are:
     <ul>
     <li> {@link TermQuery}
-    <li> {@link MultiTermQuery}
     <li> {@link BooleanQuery}
     <li> {@link WildcardQuery}
     <li> {@link PhraseQuery}
     <li> {@link PrefixQuery}
     <li> {@link MultiPhraseQuery}
     <li> {@link FuzzyQuery}
+    <li> {@link RegexpQuery}
     <li> {@link TermRangeQuery}
     <li> {@link NumericRangeQuery}
-    <li> {@link org.apache.lucene.search.spans.SpanQuery}
+    <li> {@link ConstantScoreQuery}
+    <li> {@link DisjunctionMaxQuery}
+    <li> {@link MatchAllDocsQuery}
     </ul>
+    <p>See also the family of {@link org.apache.lucene.search.spans Span Queries}
+       and additional queries available in the <a href="{@docRoot}/../queries/overview-summary.html">Queries module</a>
 */
 public abstract class Query implements Cloneable {
   private float boost = 1.0f;                     // query boost factor

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/RegexpQuery.java Sat Aug 25 10:06:07 2012
@@ -97,7 +97,9 @@ public class RegexpQuery extends Automat
       buffer.append(term.field());
       buffer.append(":");
     }
+    buffer.append('/');
     buffer.append(term.text());
+    buffer.append('/');
     buffer.append(ToStringUtils.boost(getBoost()));
     return buffer.toString();
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java Sat Aug 25 10:06:07 2012
@@ -69,6 +69,7 @@ class ReqOptSumScorer extends Scorer {
    */
   @Override
   public float score() throws IOException {
+    // TODO: sum into a double and cast to float if we ever send required clauses to BS1
     int curDoc = reqScorer.docID();
     float reqScore = reqScorer.score();
     if (optScorer == null) {

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/DataInput.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/DataInput.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/DataInput.java Sat Aug 25 10:06:07 2012
@@ -28,6 +28,13 @@ import org.apache.lucene.util.IOUtils;
 /**
  * Abstract base class for performing read operations of Lucene's low-level
  * data types.
+ *
+ * <p>{@code DataInput} may only be used from one thread, because it is not
+ * thread safe (it keeps internal state like file position). To allow
+ * multithreaded use, every {@code DataInput} instance must be cloned before
+ * used in another thread. Subclasses must therefore implement {@link #clone()},
+ * returning a new {@code DataInput} which operates on the same underlying
+ * resource, but positioned independently.
  */
 public abstract class DataInput implements Cloneable {
   /** Reads and returns a single byte.
@@ -195,12 +202,11 @@ public abstract class DataInput implemen
    */
   @Override
   public DataInput clone() {
-    DataInput clone = null;
     try {
-      clone = (DataInput)super.clone();
-    } catch (CloneNotSupportedException e) {}
-
-    return clone;
+      return (DataInput) super.clone();
+    } catch (CloneNotSupportedException e) {
+      throw new Error("This cannot happen: Failing to clone DataInput");
+    }
   }
 
   /** Reads a Map&lt;String,String&gt; previously written

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/DataOutput.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/DataOutput.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/DataOutput.java Sat Aug 25 10:06:07 2012
@@ -27,6 +27,9 @@ import org.apache.lucene.util.UnicodeUti
 /**
  * Abstract base class for performing write operations of Lucene's low-level
  * data types.
+ 
+ * <p>{@code DataOutput} may only be used from one thread, because it is not
+ * thread safe (it keeps internal state like file position).
  */
 public abstract class DataOutput {
 

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/Directory.java Sat Aug 25 10:06:07 2012
@@ -231,7 +231,7 @@ public abstract class Directory implemen
       }
       @Override
       public IndexInput openFullSlice() {
-        return (IndexInput) base.clone();
+        return base.clone();
       }
     };
   }
@@ -278,7 +278,7 @@ public abstract class Directory implemen
     
     SlicedIndexInput(final String sliceDescription, final IndexInput base, final long fileOffset, final long length, int readBufferSize) {
       super("SlicedIndexInput(" + sliceDescription + " in " + base + " slice=" + fileOffset + ":" + (fileOffset+length) + ")", readBufferSize);
-      this.base = (IndexInput) base.clone();
+      this.base = base.clone();
       this.fileOffset = fileOffset;
       this.length = length;
     }
@@ -286,7 +286,7 @@ public abstract class Directory implemen
     @Override
     public SlicedIndexInput clone() {
       SlicedIndexInput clone = (SlicedIndexInput)super.clone();
-      clone.base = (IndexInput)base.clone();
+      clone.base = base.clone();
       clone.fileOffset = fileOffset;
       clone.length = length;
       return clone;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/IndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/IndexInput.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/IndexInput.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/IndexInput.java Sat Aug 25 10:06:07 2012
@@ -22,6 +22,17 @@ import java.io.IOException;
 
 /** Abstract base class for input from a file in a {@link Directory}.  A
  * random-access input stream.  Used for all Lucene index input operations.
+ *
+ * <p>{@code IndexInput} may only be used from one thread, because it is not
+ * thread safe (it keeps internal state like file position). To allow
+ * multithreaded use, every {@code IndexInput} instance must be cloned before
+ * used in another thread. Subclasses must therefore implement {@link #clone()},
+ * returning a new {@code IndexInput} which operates on the same underlying
+ * resource, but positioned independently. Lucene never closes cloned
+ * {@code IndexInput}s, it will only do this on the original one.
+ * The original instance must take care that cloned instances throw
+ * {@link AlreadyClosedException} when the original one is closed.
+ 
  * @see Directory
  */
 public abstract class IndexInput extends DataInput implements Cloneable,Closeable {
@@ -83,4 +94,15 @@ public abstract class IndexInput extends
   public String toString() {
     return resourceDescription;
   }
+  
+  /** {@inheritDoc}
+   * <p><b>Warning:</b> Lucene never closes cloned
+   * {@code IndexInput}s, it will only do this on the original one.
+   * The original instance must take care that cloned instances throw
+   * {@link AlreadyClosedException} when the original one is closed.
+   */
+  @Override
+  public IndexInput clone() {
+    return (IndexInput) super.clone();
+  }
 }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/store/IndexOutput.java Sat Aug 25 10:06:07 2012
@@ -22,6 +22,10 @@ import java.io.IOException;
 
 /** Abstract base class for output to a file in a Directory.  A random-access
  * output stream.  Used for all Lucene index output operations.
+ 
+ * <p>{@code IndexOutput} may only be used from one thread, because it is not
+ * thread safe (it keeps internal state like file position).
+ 
  * @see Directory
  * @see IndexInput
  */

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java?rev=1377246&r1=1377245&r2=1377246&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/util/WeakIdentityMap.java Sat Aug 25 10:06:07 2012
@@ -107,6 +107,8 @@ public final class WeakIdentityMap<K,V> 
   public Iterator<K> keyIterator() {
     reap();
     final Iterator<IdentityWeakReference> iterator = backingStore.keySet().iterator();
+    // IMPORTANT: Don't use oal.util.FilterIterator here:
+    // We need *strong* reference to current key after setNext()!!!
     return new Iterator<K>() {
       // holds strong reference to next element in backing iterator:
       private Object next = null;