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/09 12:21:07 UTC

svn commit: r1371142 [24/32] - in /lucene/dev/branches/lucene3312: ./ dev-tools/ dev-tools/eclipse/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/analysis/common/ dev-tools/maven/lucene/analysis/icu/ dev-tools/maven/lucene/analysis/ku...

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Thu Aug  9 10:20:53 2012
@@ -116,9 +116,19 @@ public abstract class AtomicReader exten
   }
 
   /** Returns {@link DocsEnum} for the specified field &
-   *  term.  This may return null, if either the field or
+   *  term.  This will return null if either the field or
    *  term does not exist. */
-  public final DocsEnum termDocsEnum(Bits liveDocs, String field, BytesRef term, boolean needsFreqs) throws IOException {
+  public final DocsEnum termDocsEnum(Bits liveDocs, String field, BytesRef term) throws IOException {
+    return termDocsEnum(liveDocs, field, term, DocsEnum.FLAG_FREQS);
+  }
+
+  /** Returns {@link DocsEnum} for the specified field &
+   *  term, with control over whether freqs are required.
+   *  Some codecs may be able to optimize their
+   *  implementation when freqs are not required. This will
+   *  return null if the field or term does not
+   *  exist.  See {@link TermsEnum#docs(Bits,DocsEnum,int)}. */
+  public final DocsEnum termDocsEnum(Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
     assert field != null;
     assert term != null;
     final Fields fields = fields();
@@ -127,18 +137,30 @@ public abstract class AtomicReader exten
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator(null);
         if (termsEnum.seekExact(term, true)) {
-          return termsEnum.docs(liveDocs, null, needsFreqs);
+          return termsEnum.docs(liveDocs, null, flags);
         }
       }
     }
     return null;
   }
+  
+  /** Returns {@link DocsAndPositionsEnum} for the specified
+   *  field & term.  This will return null if the
+   *  field or term does not exist or positions weren't indexed. 
+   *  @see #termPositionsEnum(Bits, String, BytesRef, int) */
+  public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term) throws IOException {
+    return termPositionsEnum(liveDocs, field, term, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
+  }
+
 
   /** Returns {@link DocsAndPositionsEnum} for the specified
-   *  field & term.  This may return null, if either the
-   *  field or term does not exist, or needsOffsets is
-   *  true but offsets were not indexed for this field. */
-  public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term, boolean needsOffsets) throws IOException {
+   *  field & term, with control over whether offsets and payloads are
+   *  required.  Some codecs may be able to optimize their
+   *  implementation when offsets and/or payloads are not required.
+   *  This will return null if the field or term
+   *  does not exist or positions weren't indexed.  See
+   *  {@link TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}. */
+  public final DocsAndPositionsEnum termPositionsEnum(Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
     assert field != null;
     assert term != null;
     final Fields fields = fields();
@@ -147,7 +169,7 @@ public abstract class AtomicReader exten
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator(null);
         if (termsEnum.seekExact(term, true)) {
-          return termsEnum.docsAndPositions(liveDocs, null, needsOffsets);
+          return termsEnum.docsAndPositions(liveDocs, null, flags);
         }
       }
     }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReaderContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReaderContext.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReaderContext.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/AtomicReaderContext.java Thu Aug  9 10:20:53 2012
@@ -1,8 +1,5 @@
 package org.apache.lucene.index;
 
-import java.util.Collections;
-import java.util.List;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -20,6 +17,9 @@ import java.util.List;
  * limitations under the License.
  */
 
+import java.util.Collections;
+import java.util.List;
+
 /**
  * {@link IndexReaderContext} for {@link AtomicReader} instances
  * @lucene.experimental
@@ -51,8 +51,9 @@ public final class AtomicReaderContext e
   
   @Override
   public List<AtomicReaderContext> leaves() {
-    if (!isTopLevel)
+    if (!isTopLevel) {
       throw new UnsupportedOperationException("This is not a top-level context.");
+    }
     assert leaves != null;
     return leaves;
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java Thu Aug  9 10:20:53 2012
@@ -396,7 +396,8 @@ class BufferedDeletesStream {
       // System.out.println("  term=" + term);
 
       if (termsEnum.seekExact(term.bytes(), false)) {
-        DocsEnum docsEnum = termsEnum.docs(rld.getLiveDocs(), docs, false);
+        // we don't need term frequencies for this
+        DocsEnum docsEnum = termsEnum.docs(rld.getLiveDocs(), docs, 0);
         //System.out.println("BDS: got docsEnum=" + docsEnum);
 
         if (docsEnum != null) {

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Thu Aug  9 10:20:53 2012
@@ -35,6 +35,7 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.StoredDocument;
 import org.apache.lucene.index.DocValues.SortedSource;
 import org.apache.lucene.index.DocValues.Source;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
@@ -683,7 +684,6 @@ public class CheckIndex {
     DocsEnum docs = null;
     DocsEnum docsAndFreqs = null;
     DocsAndPositionsEnum postings = null;
-    DocsAndPositionsEnum offsets = null;
     
     String lastField = null;
     final FieldsEnum fieldsEnum = fields.iterator();
@@ -700,11 +700,11 @@ public class CheckIndex {
       
       // check that the field is in fieldinfos, and is indexed.
       // TODO: add a separate test to check this for different reader impls
-      FieldInfo fi = fieldInfos.fieldInfo(field);
-      if (fi == null) {
+      FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+      if (fieldInfo == null) {
         throw new RuntimeException("fieldsEnum inconsistent with fieldInfos, no fieldInfos for: " + field);
       }
-      if (!fi.isIndexed()) {
+      if (!fieldInfo.isIndexed()) {
         throw new RuntimeException("fieldsEnum inconsistent with fieldInfos, isIndexed == false for: " + field);
       }
       
@@ -756,10 +756,8 @@ public class CheckIndex {
         status.totFreq += docFreq;
         sumDocFreq += docFreq;
         
-        docs = termsEnum.docs(liveDocs, docs, false);
-        docsAndFreqs = termsEnum.docs(liveDocs, docsAndFreqs, true);
-        postings = termsEnum.docsAndPositions(liveDocs, postings, false);
-        offsets = termsEnum.docsAndPositions(liveDocs, offsets, true);
+        docs = termsEnum.docs(liveDocs, docs);
+        postings = termsEnum.docsAndPositions(liveDocs, postings);
         
         if (hasOrd) {
           long ord = -1;
@@ -780,34 +778,17 @@ public class CheckIndex {
         status.termCount++;
         
         final DocsEnum docs2;
-        final DocsEnum docsAndFreqs2;
         final boolean hasPositions;
-        final boolean hasFreqs;
-        final boolean hasOffsets;
-        if (offsets != null) {
-          docs2 = postings = offsets;
-          docsAndFreqs2 = postings = offsets;
-          hasOffsets = true;
-          hasPositions = true;
-          hasFreqs = true;
-        } else if (postings != null) {
+        // if we are checking vectors, we have freqs implicitly
+        final boolean hasFreqs = isVectors || fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+        // if we are checking vectors, offsets are a free-for-all anyway
+        final boolean hasOffsets = isVectors || fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+        if (postings != null) {
           docs2 = postings;
-          docsAndFreqs2 = postings;
-          hasOffsets = false;
           hasPositions = true;
-          hasFreqs = true;
-        } else if (docsAndFreqs != null) {
-          docs2 = docsAndFreqs;
-          docsAndFreqs2 = docsAndFreqs;
-          hasOffsets = false;
-          hasPositions = false;
-          hasFreqs = true;
         } else {
           docs2 = docs;
-          docsAndFreqs2 = null;
-          hasOffsets = false;
           hasPositions = false;
-          hasFreqs = false;
         }
         
         int lastDoc = -1;
@@ -821,7 +802,7 @@ public class CheckIndex {
           visitedDocs.set(doc);
           int freq = -1;
           if (hasFreqs) {
-            freq = docsAndFreqs2.freq();
+            freq = docs2.freq();
             if (freq <= 0) {
               throw new RuntimeException("term " + term + ": doc " + doc + ": freq " + freq + " is out of bounds");
             }
@@ -887,12 +868,12 @@ public class CheckIndex {
         }
         
         final long totalTermFreq2 = termsEnum.totalTermFreq();
-        final boolean hasTotalTermFreq = postings != null && totalTermFreq2 != -1;
+        final boolean hasTotalTermFreq = hasFreqs && totalTermFreq2 != -1;
         
         // Re-count if there are deleted docs:
         if (liveDocs != null) {
           if (hasFreqs) {
-            final DocsEnum docsNoDel = termsEnum.docs(null, docsAndFreqs, true);
+            final DocsEnum docsNoDel = termsEnum.docs(null, docsAndFreqs);
             docCount = 0;
             totalTermFreq = 0;
             while(docsNoDel.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
@@ -901,7 +882,7 @@ public class CheckIndex {
               totalTermFreq += docsNoDel.freq();
             }
           } else {
-            final DocsEnum docsNoDel = termsEnum.docs(null, docs, false);
+            final DocsEnum docsNoDel = termsEnum.docs(null, docs, 0);
             docCount = 0;
             totalTermFreq = -1;
             while(docsNoDel.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
@@ -928,7 +909,7 @@ public class CheckIndex {
         if (hasPositions) {
           for(int idx=0;idx<7;idx++) {
             final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
-            postings = termsEnum.docsAndPositions(liveDocs, postings, hasOffsets);
+            postings = termsEnum.docsAndPositions(liveDocs, postings);
             final int docID = postings.advance(skipDocID);
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
@@ -993,7 +974,7 @@ public class CheckIndex {
         } else {
           for(int idx=0;idx<7;idx++) {
             final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
-            docs = termsEnum.docs(liveDocs, docs, false);
+            docs = termsEnum.docs(liveDocs, docs, 0);
             final int docID = docs.advance(skipDocID);
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
@@ -1063,7 +1044,7 @@ public class CheckIndex {
           }
           
           int expectedDocFreq = termsEnum.docFreq();
-          DocsEnum d = termsEnum.docs(null, null, false);
+          DocsEnum d = termsEnum.docs(null, null, 0);
           int docFreq = 0;
           while (d.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
             docFreq++;
@@ -1104,7 +1085,7 @@ public class CheckIndex {
                 throw new RuntimeException("seek to existing term " + seekTerms[i] + " failed");
               }
               
-              docs = termsEnum.docs(liveDocs, docs, false);
+              docs = termsEnum.docs(liveDocs, docs, 0);
               if (docs == null) {
                 throw new RuntimeException("null DocsEnum from to existing term " + seekTerms[i]);
               }
@@ -1122,7 +1103,7 @@ public class CheckIndex {
               }
               
               totDocFreq += termsEnum.docFreq();
-              docs = termsEnum.docs(null, docs, false);
+              docs = termsEnum.docs(null, docs, 0);
               if (docs == null) {
                 throw new RuntimeException("null DocsEnum from to existing term " + seekTerms[i]);
               }
@@ -1451,6 +1432,7 @@ public class CheckIndex {
             if (crossCheckTermVectors) {
               Terms terms = tfv.terms(field);
               termsEnum = terms.iterator(termsEnum);
+              final boolean postingsHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
 
               Terms postingsTerms = postingsFields.terms(field);
               if (postingsTerms == null) {
@@ -1461,44 +1443,20 @@ public class CheckIndex {
               BytesRef term = null;
               while ((term = termsEnum.next()) != null) {
                 
-                final boolean hasPositions;
-                final boolean hasOffsets;
-                final boolean hasFreqs;
-
-                // TODO: really we need a reflection/query
-                // API so we can just ask what was indexed
-                // instead of "probing"...
+                final boolean hasProx;
 
-                // Try offsets:
-                postings = termsEnum.docsAndPositions(null, postings, true);
+                // Try positions:
+                postings = termsEnum.docsAndPositions(null, postings);
                 if (postings == null) {
-                  hasOffsets = false;
-                  // Try only positions:
-                  postings = termsEnum.docsAndPositions(null, postings, false);
-                  if (postings == null) {
-                    hasPositions = false;
-                    // Try docIDs & freqs:
-                    docs = termsEnum.docs(null, docs, true);
-                    if (docs == null) {
-                      // OK, only docIDs:
-                      hasFreqs = false;
-                      docs = termsEnum.docs(null, docs, false);
-                    } else {
-                      hasFreqs = true;
-                    }
-                  } else {
-                    hasPositions = true;
-                    hasFreqs = true;
-                  }
+                  hasProx = false;
+                  // Try docIDs & freqs:
+                  docs = termsEnum.docs(null, docs);
                 } else {
-                  hasOffsets = true;
-                  // NOTE: may be a lie... but we accept -1
-                  hasPositions = true;
-                  hasFreqs = true;
+                  hasProx = true;
                 }
 
                 final DocsEnum docs2;
-                if (hasPositions || hasOffsets) {
+                if (hasProx) {
                   assert postings != null;
                   docs2 = postings;
                 } else {
@@ -1507,30 +1465,16 @@ public class CheckIndex {
                 }
 
                 final DocsEnum postingsDocs2;
-                final boolean postingsHasFreq;
                 if (!postingsTermsEnum.seekExact(term, true)) {
                   throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
                 }
-                postingsPostings = postingsTermsEnum.docsAndPositions(null, postingsPostings, true);
+                postingsPostings = postingsTermsEnum.docsAndPositions(null, postingsPostings);
                 if (postingsPostings == null) {
-                  // Term vectors were indexed w/ offsets but postings were not
-                  postingsPostings = postingsTermsEnum.docsAndPositions(null, postingsPostings, false);
-                  if (postingsPostings == null) {
-                    postingsDocs = postingsTermsEnum.docs(null, postingsDocs, true);
-                    if (postingsDocs == null) {
-                      postingsHasFreq = false;
-                      postingsDocs = postingsTermsEnum.docs(null, postingsDocs, false);
-                      if (postingsDocs == null) {
-                        throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
-                      }
-                    } else {
-                      postingsHasFreq = true;
-                    }
-                  } else {
-                    postingsHasFreq = true;
+                  // Term vectors were indexed w/ pos but postings were not
+                  postingsDocs = postingsTermsEnum.docs(null, postingsDocs);
+                  if (postingsDocs == null) {
+                    throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
                   }
-                } else {
-                  postingsHasFreq = true;
                 }
 
                 if (postingsPostings != null) {
@@ -1550,13 +1494,13 @@ public class CheckIndex {
                   throw new RuntimeException("vector for doc " + j + " didn't return docID=0: got docID=" + doc);
                 }
 
-                if (hasFreqs) {
+                if (postingsHasFreq) {
                   final int tf = docs2.freq();
                   if (postingsHasFreq && postingsDocs2.freq() != tf) {
                     throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": freq=" + tf + " differs from postings freq=" + postingsDocs2.freq());
                   }
                 
-                  if (hasPositions || hasOffsets) {
+                  if (hasProx) {
                     for (int i = 0; i < tf; i++) {
                       int pos = postings.nextPosition();
                       if (postingsPostings != null) {
@@ -1566,32 +1510,30 @@ public class CheckIndex {
                         }
                       }
 
-                      if (hasOffsets) {
-                        // Call the methods to at least make
-                        // sure they don't throw exc:
-                        final int startOffset = postings.startOffset();
-                        final int endOffset = postings.endOffset();
-                        // TODO: these are too anal...?
-                        /*
-                          if (endOffset < startOffset) {
-                          throw new RuntimeException("vector startOffset=" + startOffset + " is > endOffset=" + endOffset);
-                          }
-                          if (startOffset < lastStartOffset) {
-                          throw new RuntimeException("vector startOffset=" + startOffset + " is < prior startOffset=" + lastStartOffset);
-                          }
-                          lastStartOffset = startOffset;
-                        */
-
-                        if (postingsPostings != null) {
-                          final int postingsStartOffset = postingsPostings.startOffset();
-
-                          final int postingsEndOffset = postingsPostings.endOffset();
-                          if (startOffset != -1 && postingsStartOffset != -1 && startOffset != postingsStartOffset) {
-                            throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": startOffset=" + startOffset + " differs from postings startOffset=" + postingsStartOffset);
-                          }
-                          if (endOffset != -1 && postingsEndOffset != -1 && endOffset != postingsEndOffset) {
-                            throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": endOffset=" + endOffset + " differs from postings endOffset=" + postingsEndOffset);
-                          }
+                      // Call the methods to at least make
+                      // sure they don't throw exc:
+                      final int startOffset = postings.startOffset();
+                      final int endOffset = postings.endOffset();
+                      // TODO: these are too anal...?
+                      /*
+                        if (endOffset < startOffset) {
+                        throw new RuntimeException("vector startOffset=" + startOffset + " is > endOffset=" + endOffset);
+                        }
+                        if (startOffset < lastStartOffset) {
+                        throw new RuntimeException("vector startOffset=" + startOffset + " is < prior startOffset=" + lastStartOffset);
+                        }
+                        lastStartOffset = startOffset;
+                      */
+
+                      if (postingsPostings != null) {
+                        final int postingsStartOffset = postingsPostings.startOffset();
+
+                        final int postingsEndOffset = postingsPostings.endOffset();
+                        if (startOffset != -1 && postingsStartOffset != -1 && startOffset != postingsStartOffset) {
+                          throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": startOffset=" + startOffset + " differs from postings startOffset=" + postingsStartOffset);
+                        }
+                        if (endOffset != -1 && postingsEndOffset != -1 && endOffset != postingsEndOffset) {
+                          throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": endOffset=" + endOffset + " differs from postings endOffset=" + postingsEndOffset);
                         }
                       }
                     }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java Thu Aug  9 10:20:53 2012
@@ -75,6 +75,9 @@ public abstract class DirectoryReader ex
    *  memory usage, at the expense of higher latency when
    *  loading a TermInfo.  The default value is 1.  Set this
    *  to -1 to skip loading the terms index entirely.
+   *  <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
+   *  implementations, including the default one in this release. It only makes
+   *  sense for terms indexes that can efficiently re-sample terms at load time.
    * @throws IOException if there is a low-level IO error
    */
   public static DirectoryReader open(final Directory directory, int termInfosIndexDivisor) throws IOException {
@@ -126,6 +129,9 @@ public abstract class DirectoryReader ex
    *  memory usage, at the expense of higher latency when
    *  loading a TermInfo.  The default value is 1.  Set this
    *  to -1 to skip loading the terms index entirely.
+   *  <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
+   *  implementations, including the default one in this release. It only makes
+   *  sense for terms indexes that can efficiently re-sample terms at load time.
    * @throws IOException if there is a low-level IO error
    */
   public static DirectoryReader open(final IndexCommit commit, int termInfosIndexDivisor) throws IOException {

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java Thu Aug  9 10:20:53 2012
@@ -334,7 +334,7 @@ public class DocTermOrds {
       final int df = te.docFreq();
       if (df <= maxTermDocFreq) {
 
-        docsEnum = te.docs(liveDocs, docsEnum, false);
+        docsEnum = te.docs(liveDocs, docsEnum, 0);
 
         // dF, but takes deletions into account
         int actualDF = 0;
@@ -668,13 +668,13 @@ public class DocTermOrds {
     }
 
     @Override    
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
-      return termsEnum.docs(liveDocs, reuse, needsFreqs);
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+      return termsEnum.docs(liveDocs, reuse, flags);
     }
 
     @Override    
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
-      return termsEnum.docsAndPositions(liveDocs, reuse, needsOffsets);
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+      return termsEnum.docsAndPositions(liveDocs, reuse, flags);
     }
 
     @Override

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocsAndPositionsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocsAndPositionsEnum.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocsAndPositionsEnum.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocsAndPositionsEnum.java Thu Aug  9 10:20:53 2012
@@ -23,7 +23,14 @@ import org.apache.lucene.util.BytesRef;
 
 /** Also iterates through positions. */
 public abstract class DocsAndPositionsEnum extends DocsEnum {
-
+  /** Flag to pass to {@link TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}
+   *  if you require offsets in the returned enum. */
+  public static final int FLAG_OFFSETS = 0x1;
+
+  /** Flag to pass to  {@link TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}
+   *  if you require payloads in the returned enum. */
+  public static final int FLAG_PAYLOADS = 0x2;
+  
   /** Returns the next position.  You should only call this
    *  up to {@link DocsEnum#freq()} times else
    *  the behavior is not defined.  If positions were not

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java Thu Aug  9 10:20:53 2012
@@ -27,6 +27,10 @@ import org.apache.lucene.util.AttributeS
  *  any of the per-doc methods. */
 public abstract class DocsEnum extends DocIdSetIterator {
 
+  /** Flag to pass to {@link TermsEnum#docs(Bits,DocsEnum,int)}
+   *  if you require term frequencies in the returned enum. */
+  public static final int FLAG_FREQS = 0x1;
+
   private AttributeSource atts = null;
 
   /** Returns term frequency in the current document.  Do

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Thu Aug  9 10:20:53 2012
@@ -202,11 +202,9 @@ final class DocumentsWriter {
    *  discarding any docs added since last flush. */
   synchronized void abort() {
     boolean success = false;
-    synchronized (this) {
-      deleteQueue.clear();
-    }
 
     try {
+      deleteQueue.clear();
       if (infoStream.isEnabled("DW")) {
         infoStream.message("DW", "abort");
       }
@@ -230,6 +228,7 @@ final class DocumentsWriter {
           perThread.unlock();
         }
       }
+      flushControl.abortPendingFlushes();
       flushControl.waitForFlush();
       success = true;
     } finally {

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Thu Aug  9 10:20:53 2012
@@ -567,19 +567,34 @@ final class DocumentsWriterFlushControl 
   }
 
   synchronized void abortFullFlushes() {
+   try {
+     abortPendingFlushes();
+   } finally {
+     fullFlush = false;
+   }
+  }
+  
+  synchronized void abortPendingFlushes() {
     try {
       for (DocumentsWriterPerThread dwpt : flushQueue) {
-        doAfterFlush(dwpt);
-        dwpt.abort();
+        try {
+          dwpt.abort();
+          doAfterFlush(dwpt);
+        } catch (Throwable ex) {
+          // ignore - keep on aborting the flush queue
+        }
       }
       for (BlockedFlush blockedFlush : blockedFlushes) {
-        flushingWriters
-            .put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
-        doAfterFlush(blockedFlush.dwpt);
-        blockedFlush.dwpt.abort();
+        try {
+          flushingWriters
+              .put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
+          blockedFlush.dwpt.abort();
+          doAfterFlush(blockedFlush.dwpt);
+        } catch (Throwable ex) {
+          // ignore - keep on aborting the blocked queue
+        }
       }
     } finally {
-      fullFlush = false;
       flushQueue.clear();
       blockedFlushes.clear();
       updateStallState();

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Thu Aug  9 10:20:53 2012
@@ -181,13 +181,13 @@ public class FilterAtomicReader extends 
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
-      return in.docs(liveDocs, reuse, needsFreqs);
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+      return in.docs(liveDocs, reuse, flags);
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
-      return in.docsAndPositions(liveDocs, reuse, needsOffsets);
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+      return in.docsAndPositions(liveDocs, reuse, flags);
     }
 
     @Override

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java Thu Aug  9 10:20:53 2012
@@ -166,13 +166,13 @@ public abstract class FilteredTermsEnum 
   }
 
   @Override
-  public DocsEnum docs(Bits bits, DocsEnum reuse, boolean needsFreqs) throws IOException {
-    return tenum.docs(bits, reuse, needsFreqs);
+  public DocsEnum docs(Bits bits, DocsEnum reuse, int flags) throws IOException {
+    return tenum.docs(bits, reuse, flags);
   }
     
   @Override
-  public DocsAndPositionsEnum docsAndPositions(Bits bits, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
-    return tenum.docsAndPositions(bits, reuse, needsOffsets);
+  public DocsAndPositionsEnum docsAndPositions(Bits bits, DocsAndPositionsEnum reuse, int flags) throws IOException {
+    return tenum.docsAndPositions(bits, reuse, flags);
   }
   
   /** This enum does not support seeking!

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Thu Aug  9 10:20:53 2012
@@ -168,7 +168,7 @@ final class FreqProxTermsWriterPerField 
       postings.lastDocCodes[termID] = docState.docID;
     } else {
       postings.lastDocCodes[termID] = docState.docID << 1;
-      postings.docFreqs[termID] = 1;
+      postings.termFreqs[termID] = 1;
       if (hasProx) {
         writeProx(termID, fieldState.position);
         if (hasOffsets) {
@@ -189,10 +189,10 @@ final class FreqProxTermsWriterPerField 
 
     FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
 
-    assert !hasFreq || postings.docFreqs[termID] > 0;
+    assert !hasFreq || postings.termFreqs[termID] > 0;
 
     if (!hasFreq) {
-      assert postings.docFreqs == null;
+      assert postings.termFreqs == null;
       if (docState.docID != postings.lastDocIDs[termID]) {
         assert docState.docID > postings.lastDocIDs[termID];
         termsHashPerField.writeVInt(0, postings.lastDocCodes[termID]);
@@ -207,13 +207,13 @@ final class FreqProxTermsWriterPerField 
 
       // Now that we know doc freq for previous doc,
       // write it & lastDocCode
-      if (1 == postings.docFreqs[termID]) {
+      if (1 == postings.termFreqs[termID]) {
         termsHashPerField.writeVInt(0, postings.lastDocCodes[termID]|1);
       } else {
         termsHashPerField.writeVInt(0, postings.lastDocCodes[termID]);
-        termsHashPerField.writeVInt(0, postings.docFreqs[termID]);
+        termsHashPerField.writeVInt(0, postings.termFreqs[termID]);
       }
-      postings.docFreqs[termID] = 1;
+      postings.termFreqs[termID] = 1;
       fieldState.maxTermFrequency = Math.max(1, fieldState.maxTermFrequency);
       postings.lastDocCodes[termID] = (docState.docID - postings.lastDocIDs[termID]) << 1;
       postings.lastDocIDs[termID] = docState.docID;
@@ -228,7 +228,7 @@ final class FreqProxTermsWriterPerField 
       }
       fieldState.uniqueTermCount++;
     } else {
-      fieldState.maxTermFrequency = Math.max(fieldState.maxTermFrequency, ++postings.docFreqs[termID]);
+      fieldState.maxTermFrequency = Math.max(fieldState.maxTermFrequency, ++postings.termFreqs[termID]);
       if (hasProx) {
         writeProx(termID, fieldState.position-postings.lastPositions[termID]);
       }
@@ -247,7 +247,7 @@ final class FreqProxTermsWriterPerField 
     public FreqProxPostingsArray(int size, boolean writeFreqs, boolean writeProx, boolean writeOffsets) {
       super(size);
       if (writeFreqs) {
-        docFreqs = new int[size];
+        termFreqs = new int[size];
       }
       lastDocIDs = new int[size];
       lastDocCodes = new int[size];
@@ -262,7 +262,7 @@ final class FreqProxTermsWriterPerField 
       //System.out.println("PA init freqs=" + writeFreqs + " pos=" + writeProx + " offs=" + writeOffsets);
     }
 
-    int docFreqs[];                                    // # times this term occurs in the current doc
+    int termFreqs[];                                   // # times this term occurs in the current doc
     int lastDocIDs[];                                  // Last docID where this term occurred
     int lastDocCodes[];                                // Code for prior doc
     int lastPositions[];                               // Last position where this term occurred
@@ -270,7 +270,7 @@ final class FreqProxTermsWriterPerField 
 
     @Override
     ParallelPostingsArray newInstance(int size) {
-      return new FreqProxPostingsArray(size, docFreqs != null, lastPositions != null, lastOffsets != null);
+      return new FreqProxPostingsArray(size, termFreqs != null, lastPositions != null, lastOffsets != null);
     }
 
     @Override
@@ -290,9 +290,9 @@ final class FreqProxTermsWriterPerField 
         assert to.lastOffsets != null;
         System.arraycopy(lastOffsets, 0, to.lastOffsets, 0, numToCopy);
       }
-      if (docFreqs != null) {
-        assert to.docFreqs != null;
-        System.arraycopy(docFreqs, 0, to.docFreqs, 0, numToCopy);
+      if (termFreqs != null) {
+        assert to.termFreqs != null;
+        System.arraycopy(termFreqs, 0, to.termFreqs, 0, numToCopy);
       }
     }
 
@@ -305,7 +305,7 @@ final class FreqProxTermsWriterPerField 
       if (lastOffsets != null) {
         bytes += RamUsageEstimator.NUM_BYTES_INT;
       }
-      if (docFreqs != null) {
+      if (termFreqs != null) {
         bytes += RamUsageEstimator.NUM_BYTES_INT;
       }
 
@@ -411,21 +411,21 @@ final class FreqProxTermsWriterPerField 
       // Now termStates has numToMerge FieldMergeStates
       // which all share the same term.  Now we must
       // interleave the docID streams.
-      int numDocs = 0;
+      int docFreq = 0;
       long totTF = 0;
       int docID = 0;
 
       while(true) {
         //System.out.println("  cycle");
-        final int termDocFreq;
+        final int termFreq;
         if (freq.eof()) {
           if (postings.lastDocCodes[termID] != -1) {
             // Return last doc
             docID = postings.lastDocIDs[termID];
             if (readTermFreq) {
-              termDocFreq = postings.docFreqs[termID];
+              termFreq = postings.termFreqs[termID];
             } else {
-              termDocFreq = -1;
+              termFreq = -1;
             }
             postings.lastDocCodes[termID] = -1;
           } else {
@@ -436,20 +436,20 @@ final class FreqProxTermsWriterPerField 
           final int code = freq.readVInt();
           if (!readTermFreq) {
             docID += code;
-            termDocFreq = -1;
+            termFreq = -1;
           } else {
             docID += code >>> 1;
             if ((code & 1) != 0) {
-              termDocFreq = 1;
+              termFreq = 1;
             } else {
-              termDocFreq = freq.readVInt();
+              termFreq = freq.readVInt();
             }
           }
 
           assert docID != postings.lastDocIDs[termID];
         }
 
-        numDocs++;
+        docFreq++;
         assert docID < state.segmentInfo.getDocCount(): "doc=" + docID + " maxDoc=" + state.segmentInfo.getDocCount();
 
         // NOTE: we could check here if the docID was
@@ -464,7 +464,7 @@ final class FreqProxTermsWriterPerField 
         // 2nd sweep does the real flush, but I suspect
         // that'd add too much time to flush.
         visitedDocs.set(docID);
-        postingsConsumer.startDoc(docID, writeTermFreq ? termDocFreq : -1);
+        postingsConsumer.startDoc(docID, writeTermFreq ? termFreq : -1);
         if (docID < delDocLimit) {
           // Mark it deleted.  TODO: we could also skip
           // writing its postings; this would be
@@ -480,7 +480,7 @@ final class FreqProxTermsWriterPerField 
           }
         }
 
-        totTF += termDocFreq;
+        totTF += termFreq;
         
         // Carefully copy over the prox + payload info,
         // changing the format to match Lucene's segment
@@ -490,7 +490,7 @@ final class FreqProxTermsWriterPerField 
           // we did record positions (& maybe payload) and/or offsets
           int position = 0;
           int offset = 0;
-          for(int j=0;j<termDocFreq;j++) {
+          for(int j=0;j<termFreq;j++) {
             final BytesRef thisPayload;
 
             if (readPositions) {
@@ -537,9 +537,9 @@ final class FreqProxTermsWriterPerField 
         }
         postingsConsumer.finishDoc();
       }
-      termsConsumer.finishTerm(text, new TermStats(numDocs, writeTermFreq ? totTF : -1));
+      termsConsumer.finishTerm(text, new TermStats(docFreq, writeTermFreq ? totTF : -1));
       sumTotalTermFreq += totTF;
-      sumDocFreq += numDocs;
+      sumDocFreq += docFreq;
     }
 
     termsConsumer.finish(writeTermFreq ? sumTotalTermFreq : -1, sumDocFreq, visitedDocs.cardinality());

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Thu Aug  9 10:20:53 2012
@@ -25,6 +25,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.regex.Matcher;
 
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.NoSuchDirectoryException;
@@ -146,57 +147,61 @@ final class IndexFileDeleter {
       // it means the directory is empty, so ignore it.
       files = new String[0];
     }
-
-    for (String fileName : files) {
-
-      if (!fileName.endsWith("write.lock") && !fileName.equals(IndexFileNames.SEGMENTS_GEN)) {
-
-        // Add this file to refCounts with initial count 0:
-        getRefCount(fileName);
-
-        if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
-
-          // This is a commit (segments or segments_N), and
-          // it's valid (<= the max gen).  Load it, then
-          // incref all files it refers to:
-          if (infoStream.isEnabled("IFD")) {
-            infoStream.message("IFD", "init: load commit \"" + fileName + "\"");
-          }
-          SegmentInfos sis = new SegmentInfos();
-          try {
-            sis.read(directory, fileName);
-          } catch (FileNotFoundException e) {
-            // LUCENE-948: on NFS (and maybe others), if
-            // you have writers switching back and forth
-            // between machines, it's very likely that the
-            // dir listing will be stale and will claim a
-            // file segments_X exists when in fact it
-            // doesn't.  So, we catch this and handle it
-            // as if the file does not exist
+    
+    if (currentSegmentsFile != null) {
+      Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
+      for (String fileName : files) {
+        m.reset(fileName);
+        if (!fileName.endsWith("write.lock") && !fileName.equals(IndexFileNames.SEGMENTS_GEN)
+            && (m.matches() || fileName.startsWith(IndexFileNames.SEGMENTS))) {
+          
+          // Add this file to refCounts with initial count 0:
+          getRefCount(fileName);
+          
+          if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+            
+            // This is a commit (segments or segments_N), and
+            // it's valid (<= the max gen).  Load it, then
+            // incref all files it refers to:
             if (infoStream.isEnabled("IFD")) {
-              infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
+              infoStream.message("IFD", "init: load commit \"" + fileName + "\"");
             }
-            sis = null;
-          } catch (IOException e) {
-            if (SegmentInfos.generationFromSegmentsFileName(fileName) <= currentGen && directory.fileLength(fileName) > 0) {
-              throw e;
-            } else {
-              // Most likely we are opening an index that
-              // has an aborted "future" commit, so suppress
-              // exc in this case
+            SegmentInfos sis = new SegmentInfos();
+            try {
+              sis.read(directory, fileName);
+            } catch (FileNotFoundException e) {
+              // LUCENE-948: on NFS (and maybe others), if
+              // you have writers switching back and forth
+              // between machines, it's very likely that the
+              // dir listing will be stale and will claim a
+              // file segments_X exists when in fact it
+              // doesn't.  So, we catch this and handle it
+              // as if the file does not exist
+              if (infoStream.isEnabled("IFD")) {
+                infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
+              }
               sis = null;
+            } catch (IOException e) {
+              if (SegmentInfos.generationFromSegmentsFileName(fileName) <= currentGen && directory.fileLength(fileName) > 0) {
+                throw e;
+              } else {
+                // Most likely we are opening an index that
+                // has an aborted "future" commit, so suppress
+                // exc in this case
+                sis = null;
+              }
             }
-          }
-          if (sis != null) {
-            final CommitPoint commitPoint = new CommitPoint(commitsToDelete, directory, sis);
-            if (sis.getGeneration() == segmentInfos.getGeneration()) {
-              currentCommitPoint = commitPoint;
-            }
-            commits.add(commitPoint);
-            incRef(sis, true);
-
-            if (lastSegmentInfos == null || sis.getGeneration() > lastSegmentInfos.getGeneration()) {
-              lastSegmentInfos = sis;
+            if (sis != null) {
+              final CommitPoint commitPoint = new CommitPoint(commitsToDelete, directory, sis);
+              if (sis.getGeneration() == segmentInfos.getGeneration()) {
+                currentCommitPoint = commitPoint;
+              }
+              commits.add(commitPoint);
+              incRef(sis, true);
+              
+              if (lastSegmentInfos == null || sis.getGeneration() > lastSegmentInfos.getGeneration()) {
+                lastSegmentInfos = sis;
+              }
             }
           }
         }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexFileNames.java Thu Aug  9 10:20:53 2012
@@ -17,6 +17,8 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.util.regex.Pattern;
+
 import org.apache.lucene.codecs.Codec;
 
 // TODO: put all files under codec and remove all the static extensions here
@@ -189,4 +191,8 @@ public final class IndexFileNames {
     }
     return filename;
   }  
+
+  // All files created by codecs much match this pattern (we
+  // check this in SegmentInfo.java):
+  static final Pattern CODEC_FILE_PATTERN = Pattern.compile("_[a-z0-9]+(_.*)?\\..*");
 }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexReaderContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexReaderContext.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexReaderContext.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexReaderContext.java Thu Aug  9 10:20:53 2012
@@ -1,7 +1,5 @@
 package org.apache.lucene.index;
 
-import java.util.List;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -19,6 +17,8 @@ import java.util.List;
  * limitations under the License.
  */
 
+import java.util.List;
+
 /**
  * A struct like class that represents a hierarchical relationship between
  * {@link IndexReader} instances. 

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Thu Aug  9 10:20:53 2012
@@ -1241,6 +1241,78 @@ public class IndexWriter implements Clos
     }
   }
 
+  /** Expert: attempts to delete by document ID, as long as
+   *  the provided reader is a near-real-time reader (from {@link
+   *  DirectoryReader#open(IndexWriter,boolean)}).  If the
+   *  provided reader is an NRT reader obtained from this
+   *  writer, and its segment has not been merged away, then
+   *  the delete succeeds and this method returns true; else, it
+   *  returns false the caller must then separately delete by
+   *  Term or Query.
+   *
+   *  <b>NOTE</b>: this method can only delete documents
+   *  visible to the currently open NRT reader.  If you need
+   *  to delete documents indexed after opening the NRT
+   *  reader you must use the other deleteDocument methods
+   *  (e.g., {@link #deleteDocuments(Term)}). */
+  public synchronized boolean tryDeleteDocument(IndexReader readerIn, int docID) throws IOException {
+
+    final AtomicReader reader;
+    if (readerIn instanceof AtomicReader) {
+      // Reader is already atomic: use the incoming docID:
+      reader = (AtomicReader) readerIn;
+    } else {
+      // Composite reader: lookup sub-reader and re-base docID:
+      List<AtomicReaderContext> leaves = readerIn.getTopReaderContext().leaves();
+      int subIndex = ReaderUtil.subIndex(docID, leaves);
+      reader = leaves.get(subIndex).reader();
+      docID -= leaves.get(subIndex).docBase;
+      assert docID >= 0;
+      assert docID < reader.maxDoc();
+    }
+
+    if (!(reader instanceof SegmentReader)) {
+      throw new IllegalArgumentException("the reader must be a SegmentReader or composite reader containing only SegmentReaders");
+    }
+      
+    final SegmentInfoPerCommit info = ((SegmentReader) reader).getSegmentInfo();
+
+    // TODO: this is a slow linear search, but, number of
+    // segments should be contained unless something is
+    // seriously wrong w/ the index, so it should be a minor
+    // cost:
+
+    if (segmentInfos.indexOf(info) != -1) {
+      ReadersAndLiveDocs rld = readerPool.get(info, false);
+      if (rld != null) {
+        synchronized(bufferedDeletesStream) {
+          rld.initWritableLiveDocs();
+          if (rld.delete(docID)) {
+            final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
+            if (fullDelCount == rld.info.info.getDocCount()) {
+              // If a merge has already registered for this
+              // segment, we leave it in the readerPool; the
+              // merge will skip merging it and will then drop
+              // it once it's done:
+              if (!mergingSegments.contains(rld.info)) {
+                segmentInfos.remove(rld.info);
+                readerPool.drop(rld.info);
+                checkpoint();
+              }
+            }
+          }
+          //System.out.println("  yes " + info.info.name + " " + docID);
+          return true;
+        }
+      } else {
+        //System.out.println("  no rld " + info.info.name + " " + docID);
+      }
+    } else {
+      //System.out.println("  no seg " + info.info.name + " " + docID);
+    }
+    return false;
+  }
+
   /**
    * Deletes the document(s) containing any of the
    * terms. All given deletes are applied and flushed atomically
@@ -2240,9 +2312,7 @@ public class IndexWriter implements Clos
         }
         SegmentInfos sis = new SegmentInfos(); // read infos from dir
         sis.read(dir);
-        final Set<String> dsFilesCopied = new HashSet<String>();
-        final Map<String, String> dsNames = new HashMap<String, String>();
-        final Set<String> copiedFiles = new HashSet<String>();
+
         for (SegmentInfoPerCommit info : sis) {
           assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
 
@@ -2255,7 +2325,7 @@ public class IndexWriter implements Clos
 
           IOContext context = new IOContext(new MergeInfo(info.info.getDocCount(), info.info.sizeInBytes(), true, -1));
           
-          infos.add(copySegmentAsIs(info, newSegName, dsNames, dsFilesCopied, context, copiedFiles));
+          infos.add(copySegmentAsIs(info, newSegName, context));
         }
       }
 
@@ -2391,25 +2461,9 @@ public class IndexWriter implements Clos
   }
 
   /** Copies the segment files as-is into the IndexWriter's directory. */
-  // TODO: this can be substantially simplified now that 3.x support/shared docstores is removed!
-  private SegmentInfoPerCommit copySegmentAsIs(SegmentInfoPerCommit info, String segName,
-                                               Map<String, String> dsNames, Set<String> dsFilesCopied, IOContext context,
-                                               Set<String> copiedFiles)
+  private SegmentInfoPerCommit copySegmentAsIs(SegmentInfoPerCommit info, String segName, IOContext context)
       throws IOException {
-    // Determine if the doc store of this segment needs to be copied. It's
-    // only relevant for segments that share doc store with others,
-    // because the DS might have been copied already, in which case we
-    // just want to update the DS name of this SegmentInfo.
-    final String dsName = info.info.name;
-    assert dsName != null;
-    final String newDsName;
-    if (dsNames.containsKey(dsName)) {
-      newDsName = dsNames.get(dsName);
-    } else {
-      dsNames.put(dsName, segName);
-      newDsName = segName;
-    }
-
+    
     // note: we don't really need this fis (its copied), but we load it up
     // so we don't pass a null value to the si writer
     FieldInfos fis = getFieldInfos(info.info);
@@ -2424,7 +2478,7 @@ public class IndexWriter implements Clos
     }
 
     //System.out.println("copy seg=" + info.info.name + " version=" + info.info.getVersion());
-    // Same SI as before but we change directory, name and docStoreSegment:
+    // Same SI as before but we change directory and name
     SegmentInfo newInfo = new SegmentInfo(directory, info.info.getVersion(), segName, info.info.getDocCount(),
                                           info.info.getUseCompoundFile(),
                                           info.info.getCodec(), info.info.getDiagnostics(), attributes);
@@ -2441,16 +2495,10 @@ public class IndexWriter implements Clos
     }
     newInfo.setFiles(segFiles);
 
-    // We must rewrite the SI file because it references
-    // segment name (its own name, if its 3.x, and doc
-    // store segment name):
+    // We must rewrite the SI file because it references segment name in its list of files, etc
     TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
-    try {
-      newInfo.getCodec().segmentInfoFormat().getSegmentInfoWriter().write(trackingDir, newInfo, fis, context);
-    } catch (UnsupportedOperationException uoe) {
-      // OK: 3x codec cannot write a new SI file;
-      // SegmentInfos will write this on commit
-    }
+
+    newInfo.getCodec().segmentInfoFormat().getSegmentInfoWriter().write(trackingDir, newInfo, fis, context);
 
     final Collection<String> siFiles = trackingDir.getCreatedFiles();
 
@@ -2465,8 +2513,7 @@ public class IndexWriter implements Clos
       }
 
       assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists; siFiles=" + siFiles;
-      assert !copiedFiles.contains(file): "file \"" + file + "\" is being copied more than once";
-      copiedFiles.add(file);
+
       info.info.dir.copy(directory, file, newFileName, context);
     }
     
@@ -3085,7 +3132,7 @@ public class IndexWriter implements Clos
     checkpoint();
 
     if (infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "after commit: " + segString());
+      infoStream.message("IW", "after commitMerge: " + segString());
     }
 
     if (merge.maxNumSegments != -1 && !dropSegment) {

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java Thu Aug  9 10:20:53 2012
@@ -392,7 +392,7 @@ public final class IndexWriterConfig ext
    * @see #setMaxBufferedDocs(int)
    * @see #setRAMBufferSizeMB(double)
    */
-  public IndexWriterConfig setFlushPolicy(FlushPolicy flushPolicy) {
+  IndexWriterConfig setFlushPolicy(FlushPolicy flushPolicy) {
     this.flushPolicy = flushPolicy;
     return this;
   }
@@ -422,7 +422,7 @@ public final class IndexWriterConfig ext
   }
   
   @Override
-  public FlushPolicy getFlushPolicy() {
+  FlushPolicy getFlushPolicy() {
     return flushPolicy;
   }
   

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java Thu Aug  9 10:20:53 2012
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat; // javadocs
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@@ -146,6 +147,29 @@ public class LiveIndexWriterConfig {
    * Takes effect immediately, but only applies to newly flushed/merged
    * segments.
    * 
+   * <p>
+   * <b>NOTE:</b> This parameter does not apply to all PostingsFormat implementations,
+   * including the default one in this release. It only makes sense for term indexes
+   * that are implemented as a fixed gap between terms. For example, 
+   * {@link Lucene40PostingsFormat} implements the term index instead based upon how
+   * terms share prefixes. To configure its parameters (the minimum and maximum size
+   * for a block), you would instead use  {@link Lucene40PostingsFormat#Lucene40PostingsFormat(int, int)}.
+   * which can also be configured on a per-field basis:
+   * <pre class="prettyprint">
+   * //customize Lucene40PostingsFormat, passing minBlockSize=50, maxBlockSize=100
+   * final PostingsFormat tweakedPostings = new Lucene40PostingsFormat(50, 100);
+   * iwc.setCodec(new Lucene40Codec() {
+   *   &#64;Override
+   *   public PostingsFormat getPostingsFormatForField(String field) {
+   *     if (field.equals("fieldWithTonsOfTerms"))
+   *       return tweakedPostings;
+   *     else
+   *       return super.getPostingsFormatForField(field);
+   *   }
+   * });
+   * </pre>
+   * Note that other implementations may have their own parameters, or no parameters at all.
+   * 
    * @see IndexWriterConfig#DEFAULT_TERM_INDEX_INTERVAL
    */
   public LiveIndexWriterConfig setTermIndexInterval(int interval) { // TODO: this should be private to the codec, not settable here
@@ -335,6 +359,10 @@ public class LiveIndexWriterConfig {
    * <p>
    * Takes effect immediately, but only applies to readers opened after this
    * call
+   * <p>
+   * <b>NOTE:</b> divisor settings &gt; 1 do not apply to all PostingsFormat
+   * implementations, including the default one in this release. It only makes
+   * sense for terms indexes that can efficiently re-sample terms at load time.
    */
   public LiveIndexWriterConfig setReaderTermsIndexDivisor(int divisor) {
     if (divisor <= 0 && divisor != -1) {
@@ -462,7 +490,7 @@ public class LiveIndexWriterConfig {
   /**
    * @see IndexWriterConfig#setFlushPolicy(FlushPolicy)
    */
-  public FlushPolicy getFlushPolicy() {
+  FlushPolicy getFlushPolicy() {
     return flushPolicy;
   }
   
@@ -497,7 +525,6 @@ public class LiveIndexWriterConfig {
     sb.append("mergePolicy=").append(getMergePolicy()).append("\n");
     sb.append("indexerThreadPool=").append(getIndexerThreadPool()).append("\n");
     sb.append("readerPooling=").append(getReaderPooling()).append("\n");
-    sb.append("flushPolicy=").append(getFlushPolicy()).append("\n");
     sb.append("perThreadHardLimitMB=").append(getRAMPerThreadHardLimitMB()).append("\n");
     return sb.toString();
   }

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Thu Aug  9 10:20:53 2012
@@ -122,32 +122,54 @@ public final class MultiFields extends F
   }
   
   /** Returns {@link DocsEnum} for the specified field &
-   *  term.  This may return null if the term does not
-   *  exist. */
-  public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, boolean needsFreqs) throws IOException {
+   *  term.  This will return null if the field or term does
+   *  not exist. */
+  public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
+    return getTermDocsEnum(r, liveDocs, field, term);
+  }
+  
+  /** Returns {@link DocsEnum} for the specified field &
+   *  term, with control over whether freqs are required.
+   *  Some codecs may be able to optimize their
+   *  implementation when freqs are not required.  This will
+   *  return null if the field or term does not exist.  See {@link
+   *  TermsEnum#docs(Bits,DocsEnum,int)}.*/
+  public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
     assert field != null;
     assert term != null;
     final Terms terms = getTerms(r, field);
     if (terms != null) {
       final TermsEnum termsEnum = terms.iterator(null);
       if (termsEnum.seekExact(term, true)) {
-        return termsEnum.docs(liveDocs, null, needsFreqs);
+        return termsEnum.docs(liveDocs, null, flags);
       }
     }
     return null;
   }
 
   /** Returns {@link DocsAndPositionsEnum} for the specified
-   *  field & term.  This may return null if the term does
-   *  not exist or positions were not indexed. */
-  public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, boolean needsOffsets) throws IOException {
+   *  field & term.  This will return null if the field or
+   *  term does not exist or positions were not indexed. 
+   *  @see #getTermPositionsEnum(IndexReader, Bits, String, BytesRef, int) */
+  public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
+    return getTermPositionsEnum(r, liveDocs, field, term, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
+  }
+
+  /** Returns {@link DocsAndPositionsEnum} for the specified
+   *  field & term, with control over whether offsets and payloads are
+   *  required.  Some codecs may be able to optimize
+   *  their implementation when offsets and/or payloads are not
+   *  required. This will return null if the field or term does not
+   *  exist or positions were not indexed. See {@link
+   *  TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}. */
+  public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
     assert field != null;
     assert term != null;
     final Terms terms = getTerms(r, field);
     if (terms != null) {
       final TermsEnum termsEnum = terms.iterator(null);
       if (termsEnum.seekExact(term, true)) {
-        return termsEnum.docsAndPositions(liveDocs, null, needsOffsets);
+        return termsEnum.docsAndPositions(liveDocs, null, flags);
       }
     }
     return null;

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Thu Aug  9 10:20:53 2012
@@ -344,7 +344,7 @@ public final class MultiTermsEnum extend
   }
 
   @Override
-  public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+  public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
     MultiDocsEnum docsEnum;
     // Can only reuse if incoming enum is also a MultiDocsEnum
     if (reuse != null && reuse instanceof MultiDocsEnum) {
@@ -394,16 +394,15 @@ public final class MultiTermsEnum extend
       }
 
       assert entry.index < docsEnum.subDocsEnum.length: entry.index + " vs " + docsEnum.subDocsEnum.length + "; " + subs.length;
-      final DocsEnum subDocsEnum = entry.terms.docs(b, docsEnum.subDocsEnum[entry.index], needsFreqs);
+      final DocsEnum subDocsEnum = entry.terms.docs(b, docsEnum.subDocsEnum[entry.index], flags);
       if (subDocsEnum != null) {
         docsEnum.subDocsEnum[entry.index] = subDocsEnum;
         subDocs[upto].docsEnum = subDocsEnum;
         subDocs[upto].slice = entry.subSlice;
         upto++;
       } else {
-        // One of our subs cannot provide freqs:
-        assert needsFreqs;
-        return null;
+        // One of our subs cannot provide a docsenum:
+        assert false;
       }
     }
 
@@ -415,7 +414,7 @@ public final class MultiTermsEnum extend
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
+  public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
     MultiDocsAndPositionsEnum docsAndPositionsEnum;
     // Can only reuse if incoming enum is also a MultiDocsAndPositionsEnum
     if (reuse != null && reuse instanceof MultiDocsAndPositionsEnum) {
@@ -466,7 +465,7 @@ public final class MultiTermsEnum extend
       }
 
       assert entry.index < docsAndPositionsEnum.subDocsAndPositionsEnum.length: entry.index + " vs " + docsAndPositionsEnum.subDocsAndPositionsEnum.length + "; " + subs.length;
-      final DocsAndPositionsEnum subPostings = entry.terms.docsAndPositions(b, docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index], needsOffsets);
+      final DocsAndPositionsEnum subPostings = entry.terms.docsAndPositions(b, docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index], flags);
 
       if (subPostings != null) {
         docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index] = subPostings;
@@ -474,7 +473,7 @@ public final class MultiTermsEnum extend
         subDocsAndPositions[upto].slice = entry.subSlice;
         upto++;
       } else {
-        if (entry.terms.docs(b, null, false) != null) {
+        if (entry.terms.docs(b, null, 0) != null) {
           // At least one of our subs does not store
           // offsets or positions -- we can't correctly
           // produce a MultiDocsAndPositions enum

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java Thu Aug  9 10:20:53 2012
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.regex.Matcher;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.store.Directory;
@@ -242,16 +243,31 @@ public final class SegmentInfo {
   private Set<String> setFiles;
 
   public void setFiles(Set<String> files) {
+    checkFileNames(files);
     setFiles = files;
     sizeInBytes = -1;
   }
 
   public void addFiles(Collection<String> files) {
+    checkFileNames(files);
     setFiles.addAll(files);
+    sizeInBytes = -1;
   }
 
   public void addFile(String file) {
+    checkFileNames(Collections.singleton(file));
     setFiles.add(file);
+    sizeInBytes = -1;
+  }
+  
+  private void checkFileNames(Collection<String> files) {
+    Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
+    for (String file : files) {
+      m.reset(file);
+      if (!m.matches()) {
+        throw new IllegalArgumentException("invalid codec filename '" + file + "', must match: " + IndexFileNames.CODEC_FILE_PATTERN.pattern());
+      }
+    }
   }
     
   /**

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java Thu Aug  9 10:20:53 2012
@@ -140,26 +140,55 @@ public abstract class TermsEnum implemen
 
   /** Get {@link DocsEnum} for the current term.  Do not
    *  call this when the enum is unpositioned.  This method
-   *  may return null (if needsFreqs is true but freqs were
-   *  not indexed for this field).
+   *  will not return null.
+   *  
+   * @param liveDocs unset bits are documents that should not
+   * be returned
+   * @param reuse pass a prior DocsEnum for possible reuse */
+  public final DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
+    return docs(liveDocs, reuse, DocsEnum.FLAG_FREQS);
+  }
+
+  /** Get {@link DocsEnum} for the current term, with
+   *  control over whether freqs are required.  Do not
+   *  call this when the enum is unpositioned.  This method
+   *  will not return null.
    *  
    * @param liveDocs unset bits are documents that should not
    * be returned
    * @param reuse pass a prior DocsEnum for possible reuse
-   * @param needsFreqs true if the caller intends to call
-   * {@link DocsEnum#freq}.  If you pass false you must not
-   * call {@link DocsEnum#freq} in the returned DocsEnum. */
-  public abstract DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException;
+   * @param flags specifies which optional per-document values
+   *        you require; see {@link DocsEnum#FLAG_FREQS} 
+   * @see #docs(Bits, DocsEnum, int) */
+  public abstract DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException;
 
   /** Get {@link DocsAndPositionsEnum} for the current term.
-   *  Do not call this when the enum is unpositioned.
-   *  This method will only return null if needsOffsets is
-   *  true but offsets were not indexed.
+   *  Do not call this when the enum is unpositioned.  This
+   *  method will return null if positions were not
+   *  indexed.
+   *  
+   *  @param liveDocs unset bits are documents that should not
+   *  be returned
+   *  @param reuse pass a prior DocsAndPositionsEnum for possible reuse
+   *  @see #docsAndPositions(Bits, DocsAndPositionsEnum, int) */
+  public final DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
+    return docsAndPositions(liveDocs, reuse, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
+  }
+
+  /** Get {@link DocsAndPositionsEnum} for the current term,
+   *  with control over whether offsets and payloads are
+   *  required.  Some codecs may be able to optimize their
+   *  implementation when offsets and/or payloads are not required.
+   *  Do not call this when the enum is unpositioned.  This
+   *  will return null if positions were not indexed.
+
    *  @param liveDocs unset bits are documents that should not
    *  be returned
    *  @param reuse pass a prior DocsAndPositionsEnum for possible reuse
-   *  @param needsOffsets true if offsets are required */
-  public abstract DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException;
+   *  @param flags specifies which optional per-position values you
+   *         require; see {@link DocsAndPositionsEnum#FLAG_OFFSETS} and 
+   *         {@link DocsAndPositionsEnum#FLAG_PAYLOADS}. */
+  public abstract DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException;
 
   /**
    * Expert: Returns the TermsEnums internal state to position the TermsEnum
@@ -220,12 +249,12 @@ public abstract class TermsEnum implemen
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) {
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
       throw new IllegalStateException("this method should never be called");
     }
       
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) {
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
       throw new IllegalStateException("this method should never be called");
     }
       

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java Thu Aug  9 10:20:53 2012
@@ -361,11 +361,9 @@ public class BooleanQuery extends Query 
         final Scorer scorer = weight.scorer(context, true, false, acceptDocs);
         if (scorer == null) {
           return null;
-        }
-        if (scorer instanceof TermScorer) {
-          docsAndFreqs[i] = new DocsAndFreqs((TermScorer) scorer);
         } else {
-          docsAndFreqs[i] = new DocsAndFreqs((MatchOnlyTermScorer) scorer);
+          assert scorer instanceof TermScorer;
+          docsAndFreqs[i] = new DocsAndFreqs((TermScorer) scorer);
         }
       }
       return new ConjunctionTermScorer(this, disableCoord ? 1.0f : coord(

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1371142&r1=1371141&r2=1371142&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java Thu Aug  9 10:20:53 2012
@@ -177,7 +177,7 @@ class BooleanScorer2 extends Scorer {
                                               List<Scorer> requiredScorers) throws IOException {
     // each scorer from the list counted as a single matcher
     final int requiredNrMatchers = requiredScorers.size();
-    return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(requiredScorers.size(), requiredScorers.size()), requiredScorers) {
+    return new ConjunctionScorer(weight, requiredScorers) {
       private int lastScoredDoc = -1;
       // Save the score of lastScoredDoc, so that we don't compute it more than
       // once in score().
@@ -201,8 +201,8 @@ class BooleanScorer2 extends Scorer {
   }
 
   private Scorer dualConjunctionSumScorer(boolean disableCoord,
-                                          Scorer req1, Scorer req2) throws IOException { // non counting.
-    return new ConjunctionScorer(weight, disableCoord ? 1.0f : ((BooleanWeight)weight).coord(2, 2), req1, req2);
+                                                Scorer req1, Scorer req2) throws IOException { // non counting.
+    return new ConjunctionScorer(weight, req1, req2);
     // All scorers match, so defaultSimilarity always has 1 as
     // the coordination factor.
     // Therefore the sum of the scores of two scorers

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=1371142&r1=1371141&r2=1371142&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 Thu Aug  9 10:20:53 2012
@@ -27,17 +27,15 @@ import java.util.Comparator;
 class ConjunctionScorer extends Scorer {
   
   private final Scorer[] scorers;
-  private final float coord;
   private int lastDoc = -1;
 
-  public ConjunctionScorer(Weight weight, float coord, Collection<Scorer> scorers) throws IOException {
-    this(weight, coord, scorers.toArray(new Scorer[scorers.size()]));
+  public ConjunctionScorer(Weight weight, Collection<Scorer> scorers) throws IOException {
+    this(weight, scorers.toArray(new Scorer[scorers.size()]));
   }
 
-  public ConjunctionScorer(Weight weight, float coord, Scorer... scorers) throws IOException {
+  public ConjunctionScorer(Weight weight, Scorer... scorers) throws IOException {
     super(weight);
     this.scorers = scorers;
-    this.coord = coord;
     
     for (int i = 0; i < scorers.length; i++) {
       if (scorers[i].nextDoc() == NO_MORE_DOCS) {
@@ -135,7 +133,7 @@ class ConjunctionScorer extends Scorer {
     for (int i = 0; i < scorers.length; i++) {
       sum += scorers[i].score();
     }
-    return sum * coord;
+    return sum;
   }
 
   @Override

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=1371142&r1=1371141&r2=1371142&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 Thu Aug  9 10:20:53 2012
@@ -121,10 +121,6 @@ class ConjunctionTermScorer extends Scor
       this(termScorer, termScorer.getDocsEnum(), termScorer.getDocFreq());
     }
     
-    DocsAndFreqs(MatchOnlyTermScorer termScorer) {
-      this(termScorer, termScorer.getDocsEnum(), termScorer.getDocFreq());
-    }
-    
     DocsAndFreqs(Scorer scorer, DocsEnum docs, int docFreq) {
       this.docs = docs;
       this.docFreq = docFreq;

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=1371142&r1=1371141&r2=1371142&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 Thu Aug  9 10:20:53 2012
@@ -364,7 +364,7 @@ class FieldCacheImpl implements FieldCac
               break;
             }
             final byte termval = parser.parseByte(term);
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -437,7 +437,7 @@ class FieldCacheImpl implements FieldCac
               break;
             }
             final short termval = parser.parseShort(term);
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -541,7 +541,7 @@ class FieldCacheImpl implements FieldCac
               retArray = new int[maxDoc];
             }
 
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -608,7 +608,7 @@ class FieldCacheImpl implements FieldCac
             res = new FixedBitSet(maxDoc);
           }
 
-          docs = termsEnum.docs(null, docs, false);
+          docs = termsEnum.docs(null, docs, 0);
           // TODO: use bulk API
           while (true) {
             final int docID = docs.nextDoc();
@@ -691,7 +691,7 @@ class FieldCacheImpl implements FieldCac
               retArray = new float[maxDoc];
             }
             
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -779,7 +779,7 @@ class FieldCacheImpl implements FieldCac
               retArray = new long[maxDoc];
             }
 
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -868,7 +868,7 @@ class FieldCacheImpl implements FieldCac
               retArray = new double[maxDoc];
             }
 
-            docs = termsEnum.docs(null, docs, false);
+            docs = termsEnum.docs(null, docs, 0);
             while (true) {
               final int docID = docs.nextDoc();
               if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -1042,12 +1042,12 @@ class FieldCacheImpl implements FieldCac
       }
 
       @Override
-      public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+      public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
         throw new UnsupportedOperationException();
       }
 
       @Override
-      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
+      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
         throw new UnsupportedOperationException();
       }
 
@@ -1167,7 +1167,7 @@ class FieldCacheImpl implements FieldCac
             termOrdToBytesOffset = termOrdToBytesOffset.resize(ArrayUtil.oversize(1+termOrd, 1));
           }
           termOrdToBytesOffset.set(termOrd, bytes.copyUsingLengthPrefix(term));
-          docs = termsEnum.docs(null, docs, false);
+          docs = termsEnum.docs(null, docs, 0);
           while (true) {
             final int docID = docs.nextDoc();
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -1288,7 +1288,7 @@ class FieldCacheImpl implements FieldCac
             break;
           }
           final long pointer = bytes.copyUsingLengthPrefix(term);
-          docs = termsEnum.docs(null, docs, false);
+          docs = termsEnum.docs(null, docs, 0);
           while (true) {
             final int docID = docs.nextDoc();
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {