You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/04/14 21:12:10 UTC

svn commit: r1673526 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/backward-codecs/ lucene/backward-codecs/src/test/org/apache/lucene/index/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/blocktree/ lucene/core/src/java/org/apache/...

Author: mikemccand
Date: Tue Apr 14 19:12:09 2015
New Revision: 1673526

URL: http://svn.apache.org/r1673526
Log:
LUCENE-6382: enforce max allowed indexed position

Added:
    lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/index/TestMaxPositionInOldIndex.java
      - copied unchanged from r1673508, lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/index/TestMaxPositionInOldIndex.java
    lucene/dev/branches/branch_5x/lucene/backward-codecs/src/test/org/apache/lucene/index/maxposindex.zip
      - copied unchanged from r1673508, lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/index/maxposindex.zip
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestMaxPosition.java
      - copied unchanged from r1673508, lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestMaxPosition.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/backward-codecs/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/Version.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
    lucene/dev/branches/branch_5x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Tue Apr 14 19:12:09 2015
@@ -84,6 +84,9 @@ Other
 * LUCENE-6399: Benchmark module's QueryMaker.resetInputs should call setConfig
   so queries can react to property changes in new rounds. (David Smiley)
 
+* LUCENE-6382: Lucene now enforces that positions never exceed the
+  maximum value IndexWriter.MAX_POSITION.  (Robert Muir, Mike McCandless)
+
 Build
 
 * LUCENE-6420: Update forbiddenapis to v1.8  (Uwe Schindler)

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/AutoPrefixTermsWriter.java Tue Apr 14 19:12:09 2015
@@ -211,12 +211,17 @@ class AutoPrefixTermsWriter {
       }
     }
 
+    // Even though we visited terms in already-sorted order, the prefixes
+    // can be slightly unsorted, e.g. aaaaa will be before aaa, so we
+    // must sort here so our caller can do merge sort into actual terms
+    // when writing.  Probably we should use CollectionUtil.timSort here?
     Collections.sort(prefixes);
   }
 
   /** Pushes the new term to the top of the stack, and writes new blocks. */
   private void pushTerm(BytesRef text) throws IOException {
     int limit = Math.min(lastTerm.length(), text.length);
+    //if (DEBUG) System.out.println("\nterm: " + text.utf8ToString());
 
     // Find common prefix between last term and current term:
     int pos = 0;
@@ -234,10 +239,10 @@ class AutoPrefixTermsWriter {
       int prefixTopSize = pending.size() - prefixStarts[i];
 
       while (prefixTopSize >= minItemsInPrefix) {       
-        //if (DEBUG) System.out.println("pushTerm i=" + i + " prefixTopSize=" + prefixTopSize + " minItemsInBlock=" + minItemsInPrefix);
+        //if (DEBUG) System.out.println("  pop: i=" + i + " prefixTopSize=" + prefixTopSize + " minItemsInBlock=" + minItemsInPrefix);
         savePrefixes(i+1, prefixTopSize);
         //prefixStarts[i] -= prefixTopSize;
-        //System.out.println("    after savePrefixes: " + (pending.size() - prefixStarts[i]) + " pending.size()=" + pending.size() + " start=" + prefixStarts[i]);
+        //if (DEBUG) System.out.println("    after savePrefixes: " + (pending.size() - prefixStarts[i]) + " pending.size()=" + pending.size() + " start=" + prefixStarts[i]);
 
         // For large floor blocks, it's possible we should now re-run on the new prefix terms we just created:
         prefixTopSize = pending.size() - prefixStarts[i];
@@ -267,27 +272,52 @@ class AutoPrefixTermsWriter {
 
     assert count > 0;
 
-    //if (DEBUG2) {
-    //  BytesRef br = new BytesRef(lastTerm.bytes());
-    //  br.length = prefixLength;
-    //  System.out.println("  savePrefixes: seg=" + segment + " " + brToString(br) + " count=" + count + " pending.size()=" + pending.size());
-    //}
+    /*
+    if (DEBUG2) {
+      BytesRef br = new BytesRef(lastTerm.bytes());
+      br.length = prefixLength;
+      //System.out.println("  savePrefixes: seg=" + segment + " " + brToString(br) + " count=" + count + " pending.size()=" + pending.size());
+      System.out.println("  savePrefixes: " + brToString(br) + " count=" + count + " pending.size()=" + pending.size());
+    }
+    */
 
     int lastSuffixLeadLabel = -2;
 
     int start = pending.size()-count;
     assert start >=0;
 
+    // Special case empty-string suffix case: we are being asked to build prefix terms for all aaa* terms, but 
+    // the exact term aaa is here, and we must skip it (it is handled "higher", under the aa* terms):
+    Object o = pending.get(start);
+    boolean skippedEmptyStringSuffix = false;
+    if (o instanceof byte[]) {
+      if (((byte[]) o).length == prefixLength) {
+        start++;
+        count--;
+        //if (DEBUG) System.out.println("  skip empty-string term suffix");
+        skippedEmptyStringSuffix = true;
+      }
+    } else {
+      PrefixTerm prefix = (PrefixTerm) o;
+      if (prefix.term.bytes.length == prefixLength) {
+        start++;
+        count--;
+        //if (DEBUG) System.out.println("  skip empty-string PT suffix");
+        skippedEmptyStringSuffix = true;
+      }
+    }
+
     int end = pending.size();
     int nextBlockStart = start;
     int nextFloorLeadLabel = -1;
     int prefixCount = 0;
-    int pendingCount = 0;
+
     PrefixTerm lastPTEntry = null;
+
     for (int i=start; i<end; i++) {
 
       byte[] termBytes;
-      Object o = pending.get(i);
+      o = pending.get(i);
       PrefixTerm ptEntry;
       if (o instanceof byte[]) {
         ptEntry = null;
@@ -300,23 +330,15 @@ class AutoPrefixTermsWriter {
           ptEntry = null;
         }
       }
-      pendingCount++;
 
-      //if (DEBUG) System.out.println("    check term=" + brToString(new BytesRef(termBytes)));
+      //if (DEBUG) System.out.println("    check term=" + brToString(new BytesRef(termBytes)) + " o=" + o);
 
-      int suffixLeadLabel;
+      // We handled the empty-string suffix case up front:
+      assert termBytes.length > prefixLength;
 
-      if (termBytes.length == prefixLength) {
-        // Suffix is 0, i.e. prefix 'foo' and term is
-        // 'foo' so the term has empty string suffix
-        // in this block
-        assert lastSuffixLeadLabel == -2;
-        suffixLeadLabel = -2;
-      } else {
-        suffixLeadLabel = termBytes[prefixLength] & 0xff;
-      }
+      int suffixLeadLabel = termBytes[prefixLength] & 0xff;
 
-      // if (DEBUG) System.out.println("  i=" + i + " ent=" + ent + " suffixLeadLabel=" + suffixLeadLabel);
+      //if (DEBUG) System.out.println("  i=" + i + " o=" + o + " suffixLeadLabel=" + Integer.toHexString(suffixLeadLabel) + " pendingCount=" + (i - nextBlockStart) + " min=" + minItemsInPrefix);
 
       if (suffixLeadLabel != lastSuffixLeadLabel) {
         // This is a boundary, a chance to make an auto-prefix term if we want:
@@ -327,8 +349,9 @@ class AutoPrefixTermsWriter {
         // than the lead start of the current entry:
         assert suffixLeadLabel > lastSuffixLeadLabel: "suffixLeadLabel=" + suffixLeadLabel + " vs lastSuffixLeadLabel=" + lastSuffixLeadLabel;
 
-        // NOTE: must check nextFloorLeadLabel in case minItemsInPrefix is 2 and prefix is 'a' and we've seen 'a' and then 'aa'
-        if (pendingCount >= minItemsInPrefix && end-nextBlockStart > maxItemsInPrefix && nextFloorLeadLabel != -1) {
+        int itemsInBlock = i - nextBlockStart;
+
+        if (itemsInBlock >= minItemsInPrefix && end-nextBlockStart > maxItemsInPrefix) {
           // The count is too large for one block, so we must break it into "floor" blocks, where we record
           // the leading label of the suffix of the first term in each floor block, so at search time we can
           // jump to the right floor block.  We just use a naive greedy segmenter here: make a new floor
@@ -338,11 +361,10 @@ class AutoPrefixTermsWriter {
           // If the last entry was another prefix term of the same length, then it represents a range of terms, so we must use its ending
           // prefix label as our ending label:
           if (lastPTEntry != null) {
+            //if (DEBUG) System.out.println("  use last");
             lastSuffixLeadLabel = lastPTEntry.floorLeadEnd;
           }
-
           savePrefix(prefixLength, nextFloorLeadLabel, lastSuffixLeadLabel);
-          pendingCount = 0;
 
           prefixCount++;
           nextFloorLeadLabel = suffixLeadLabel;
@@ -356,6 +378,7 @@ class AutoPrefixTermsWriter {
 
         lastSuffixLeadLabel = suffixLeadLabel;
       }
+
       lastPTEntry = ptEntry;
     }
 
@@ -370,6 +393,12 @@ class AutoPrefixTermsWriter {
         if (prefixLength > 0) {
           savePrefix(prefixLength, -2, 0xff);
           prefixCount++;
+          
+          // If we skipped empty string suffix, e.g. term aaa for prefix aaa*, since we
+          // are now writing the full aaa* prefix term, we include it here:
+          if (skippedEmptyStringSuffix) {
+            count++;
+          }
         } else {
           // Don't add a prefix term for all terms in the index!
         }
@@ -384,16 +413,8 @@ class AutoPrefixTermsWriter {
     }
 
     // Remove slice from the top of the pending stack, that we just wrote:
-    int sizeToClear = count;
-    if (prefixCount > 1) {
-      Object o = pending.get(pending.size()-count);
-      if (o instanceof byte[] && ((byte[]) o).length == prefixLength) {
-        // If we were just asked to write all f* terms, but there were too many and so we made floor blocks, the exact term 'f' will remain
-        // as its own item, followed by floor block terms like f[a-m]*, f[n-z]*, so in this case we leave 3 (not 2) items on the pending stack:
-        sizeToClear--;
-      }
-    }
-    pending.subList(pending.size()-sizeToClear, pending.size()).clear();
+
+    pending.subList(pending.size()-count, pending.size()).clear();
 
     // Append prefix terms for each prefix, since these count like real terms that also need to be "rolled up":
     for(int i=0;i<prefixCount;i++) {
@@ -410,6 +431,8 @@ class AutoPrefixTermsWriter {
 
     PrefixTerm pt = new PrefixTerm(prefix, floorLeadStart, floorLeadEnd); 
     //if (DEBUG2) System.out.println("    savePrefix: seg=" + segment + " " + pt + " count=" + count);
+    //if (DEBUG) System.out.println("    savePrefix: " + pt);
+
     prefixes.add(pt);
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsWriter.java Tue Apr 14 19:12:09 2015
@@ -36,6 +36,7 @@ import org.apache.lucene.codecs.lucene50
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
@@ -250,6 +251,12 @@ public final class Lucene50PostingsWrite
 
   @Override
   public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
+    if (position > IndexWriter.MAX_POSITION) {
+      throw new CorruptIndexException("position=" + position + " is too large (> IndexWriter.MAX_POSITION=" + IndexWriter.MAX_POSITION + ")", docOut);
+    }
+    if (position < 0) {
+      throw new CorruptIndexException("position=" + position + " is < 0", docOut);
+    }
     posDeltaBuffer[posBufferUpto] = position - lastPosition;
     if (writePayloads) {
       if (payload == null || payload.length == 0) {

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Tue Apr 14 19:12:09 2015
@@ -217,6 +217,7 @@ public abstract class PerFieldPostingsFo
 
     private final Map<String,FieldsProducer> fields = new TreeMap<>();
     private final Map<String,FieldsProducer> formats = new HashMap<>();
+    private final String segment;
     
     // clone for merge
     FieldsReader(FieldsReader other) throws IOException {
@@ -234,6 +235,8 @@ public abstract class PerFieldPostingsFo
         assert producer != null;
         fields.put(ent.getKey(), producer);
       }
+
+      segment = other.segment;
     }
 
     public FieldsReader(final SegmentReadState readState) throws IOException {
@@ -267,6 +270,8 @@ public abstract class PerFieldPostingsFo
           IOUtils.closeWhileHandlingException(formats.values());
         }
       }
+
+      this.segment = readState.segmentInfo.name;
     }
 
     @Override
@@ -320,7 +325,7 @@ public abstract class PerFieldPostingsFo
 
     @Override
     public String toString() {
-      return "PerFieldPostings(formats=" + formats.size() + ")";
+      return "PerFieldPostings(segment=" + segment + " formats=" + formats.size() + ")";
     }
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Tue Apr 14 19:12:09 2015
@@ -1341,6 +1341,9 @@ public class CheckIndex implements Close
               if (pos < 0) {
                 throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + " is out of bounds");
               }
+              if (pos > IndexWriter.MAX_POSITION) {
+                throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + " > IndexWriter.MAX_POSITION=" + IndexWriter.MAX_POSITION);
+              }
               if (pos < lastPos) {
                 throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + " < lastPos " + lastPos);
               }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java Tue Apr 14 19:12:09 2015
@@ -623,8 +623,11 @@ final class DefaultIndexingChain extends
           if (invertState.position < invertState.lastPosition) {
             if (posIncr == 0) {
               throw new IllegalArgumentException("first position increment must be > 0 (got 0) for field '" + field.name() + "'");
+            } else {
+              throw new IllegalArgumentException("position increments (and gaps) must be >= 0 (got " + posIncr + ") for field '" + field.name() + "'");
             }
-            throw new IllegalArgumentException("position increments (and gaps) must be >= 0 (got " + posIncr + ") for field '" + field.name() + "'");
+          } else if (invertState.position > IndexWriter.MAX_POSITION) {
+            throw new IllegalArgumentException("position " + invertState.position + " is too large for field '" + field.name() + "': max allowed position is " + IndexWriter.MAX_POSITION);
           }
           invertState.lastPosition = invertState.position;
           if (posIncr == 0) {

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Tue Apr 14 19:12:09 2015
@@ -203,6 +203,9 @@ public class IndexWriter implements Clos
   // ArrayUtil.MAX_ARRAY_LENGTH here because this can vary across JVMs:
   public static final int MAX_DOCS = Integer.MAX_VALUE - 128;
 
+  /** Maximum value of the token position in an indexed field. */
+  public static final int MAX_POSITION = Integer.MAX_VALUE - 128;
+
   // Use package-private instance var to enforce the limit so testing
   // can use less electricity:
   private static int actualMaxDocs = MAX_DOCS;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java Tue Apr 14 19:12:09 2015
@@ -46,21 +46,23 @@ public class MappedMultiFields extends F
     if (terms == null) {
       return null;
     } else {
-      return new MappedMultiTerms(mergeState, terms);
+      return new MappedMultiTerms(field, mergeState, terms);
     }
   }
 
   private static class MappedMultiTerms extends FilterTerms {
     final MergeState mergeState;
+    final String field;
 
-    public MappedMultiTerms(MergeState mergeState, MultiTerms multiTerms) {
+    public MappedMultiTerms(String field, MergeState mergeState, MultiTerms multiTerms) {
       super(multiTerms);
+      this.field = field;
       this.mergeState = mergeState;
     }
 
     @Override
     public TermsEnum iterator() throws IOException {
-      return new MappedMultiTermsEnum(mergeState, (MultiTermsEnum) in.iterator());
+      return new MappedMultiTermsEnum(field, mergeState, (MultiTermsEnum) in.iterator());
     }
 
     @Override
@@ -86,9 +88,11 @@ public class MappedMultiFields extends F
 
   private static class MappedMultiTermsEnum extends FilterTermsEnum {
     final MergeState mergeState;
+    final String field;
 
-    public MappedMultiTermsEnum(MergeState mergeState, MultiTermsEnum multiTermsEnum) {
+    public MappedMultiTermsEnum(String field, MergeState mergeState, MultiTermsEnum multiTermsEnum) {
       super(multiTermsEnum);
+      this.field = field;
       this.mergeState = mergeState;
     }
 
@@ -110,26 +114,19 @@ public class MappedMultiFields extends F
 
       MappingMultiPostingsEnum mappingDocsAndPositionsEnum;
       if (reuse instanceof MappingMultiPostingsEnum) {
-        mappingDocsAndPositionsEnum = (MappingMultiPostingsEnum) reuse;
+        MappingMultiPostingsEnum postings = (MappingMultiPostingsEnum) reuse;
+        if (postings.field.equals(this.field)) {
+          mappingDocsAndPositionsEnum = postings;
+        } else {
+          mappingDocsAndPositionsEnum = new MappingMultiPostingsEnum(field, mergeState);
+        }
       } else {
-        mappingDocsAndPositionsEnum = new MappingMultiPostingsEnum(mergeState);
+        mappingDocsAndPositionsEnum = new MappingMultiPostingsEnum(field, mergeState);
       }
 
       MultiPostingsEnum docsAndPositionsEnum = (MultiPostingsEnum) in.postings(liveDocs, mappingDocsAndPositionsEnum.multiDocsAndPositionsEnum, flags);
       mappingDocsAndPositionsEnum.reset(docsAndPositionsEnum);
       return mappingDocsAndPositionsEnum;
-
-/*
-      MappingMultiDocsEnum mappingDocsEnum;
-      if (reuse instanceof MappingMultiDocsEnum) {
-        mappingDocsEnum = (MappingMultiDocsEnum) reuse;
-      } else {
-        mappingDocsEnum = new MappingMultiDocsEnum(mergeState);
-      }
-      
-      MultiDocsEnum docsEnum = (MultiDocsEnum) in.docs(liveDocs, mappingDocsEnum.multiDocsEnum, flags);
-      mappingDocsEnum.reset(docsEnum);
-      return mappingDocsEnum;*/
     }
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java Tue Apr 14 19:12:09 2015
@@ -39,9 +39,11 @@ final class MappingMultiPostingsEnum ext
   int doc = -1;
   private MergeState mergeState;
   MultiPostingsEnum multiDocsAndPositionsEnum;
+  final String field;
 
   /** Sole constructor. */
-  public MappingMultiPostingsEnum(MergeState mergeState) {
+  public MappingMultiPostingsEnum(String field, MergeState mergeState) {
+    this.field = field;
     this.mergeState = mergeState;
   }
 
@@ -112,9 +114,17 @@ final class MappingMultiPostingsEnum ext
 
   @Override
   public int nextPosition() throws IOException {
-    return current.nextPosition();
+    int pos = current.nextPosition();
+    if (pos < 0) {
+      throw new CorruptIndexException("position=" + pos + " is negative, field=\"" + field + " doc=" + doc,
+                                      mergeState.fieldsProducers[upto].toString());
+    } else if (pos > IndexWriter.MAX_POSITION) {
+      throw new CorruptIndexException("position=" + pos + " is too large (> IndexWriter.MAX_POSITION=" + IndexWriter.MAX_POSITION + "), field=\"" + field + "\" doc=" + doc,
+                                      mergeState.fieldsProducers[upto].toString());
+    }
+    return pos;
   }
-
+  
   @Override
   public int startOffset() throws IOException {
     return current.startOffset();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/Version.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/Version.java Tue Apr 14 19:12:09 2015
@@ -417,7 +417,7 @@ public final class Version {
   /** Returns a new version based on raw numbers
    *
    *  @lucene.internal */
-  public static final Version fromBits(int major, int minor, int bugfix) {
+  public static Version fromBits(int major, int minor, int bugfix) {
     return new Version(major, minor, bugfix);
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java Tue Apr 14 19:12:09 2015
@@ -32,6 +32,7 @@ package org.apache.lucene.util.automaton
 import java.util.*;
 
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
 
 /**
  * Construction of basic automata.
@@ -216,6 +217,16 @@ final public class Automata {
     return s;
   }
 
+  private static boolean suffixIsZeros(BytesRef br, int len) {
+    for(int i=len;i<br.length;i++) {
+      if (br.bytes[br.offset+i] != 0) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
   /** Creates a new deterministic, minimal automaton accepting
    *  all binary terms in the specified interval.  Note that unlike
    *  {@link #makeDecimalInterval}, the returned automaton is infinite,
@@ -238,17 +249,6 @@ final public class Automata {
       minInclusive = true;
     }
 
-    // Empty string corner cases:
-    if (max != null && maxInclusive == false && max.length == 1 && max.bytes[max.offset] == 0) {
-      max = new BytesRef();
-      maxInclusive = true;
-    }
-
-    if (min != null && minInclusive == false && min.length == 0) {
-      min = new BytesRef(new byte[1]);
-      minInclusive = true;
-    }
-
     int cmp;
     if (max != null) {
       cmp = min.compareTo(max);
@@ -270,8 +270,56 @@ final public class Automata {
       return makeEmpty();
     }
 
+    if (max != null &&
+        StringHelper.startsWith(max, min) &&
+        suffixIsZeros(max, min.length)) {
+
+      // Finite case: no sink state!
+
+      int maxLength = max.length;
+
+      // the == case was handled above
+      assert maxLength > min.length;
+
+      //  bar -> bar\0+
+      if (maxInclusive == false) {
+        maxLength--;
+      }
+
+      if (maxLength == min.length) {
+        if (minInclusive == false) {
+          return makeEmpty();
+        } else {
+          return makeBinary(min);
+        }
+      }
+
+      Automaton a = new Automaton();
+      int lastState = a.createState();
+      for (int i=0;i<min.length;i++) {
+        int state = a.createState();
+        int label = min.bytes[min.offset+i] & 0xff;
+        a.addTransition(lastState, state, label);
+        lastState = state;
+      }
+
+      if (minInclusive) {
+        a.setAccept(lastState, true);
+      }
+
+      for(int i=min.length;i<maxLength;i++) {
+        int state = a.createState();
+        a.addTransition(lastState, state, 0);
+        a.setAccept(state, true);
+        lastState = state;
+      }
+      a.finishState();
+      return a;
+    }
+
     Automaton a = new Automaton();
     int startState = a.createState();
+
     int sinkState = a.createState();
     a.setAccept(sinkState, true);
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/Test2BPositions.java Tue Apr 14 19:12:09 2015
@@ -29,12 +29,9 @@ import org.apache.lucene.store.BaseDirec
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.TimeUnits;
 import org.apache.lucene.util.LuceneTestCase.Monster;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 
-import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
-
 /**
  * Test indexes ~82M docs with 52 positions each, so you get &gt; Integer.MAX_VALUE positions
  * @lucene.experimental
@@ -60,8 +57,8 @@ public class Test2BPositions extends Luc
 
     MergePolicy mp = w.getConfig().getMergePolicy();
     if (mp instanceof LogByteSizeMergePolicy) {
-     // 1 petabyte:
-     ((LogByteSizeMergePolicy) mp).setMaxMergeMB(1024*1024*1024);
+      // 1 petabyte:
+      ((LogByteSizeMergePolicy) mp).setMaxMergeMB(1024*1024*1024);
     }
 
     Document doc = new Document();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Tue Apr 14 19:12:09 2015
@@ -31,10 +31,8 @@ import java.util.Random;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -1728,49 +1726,7 @@ public class TestIndexWriterExceptions e
     uoe.doFail = false;
     d.close();
   }
-  
-  public void testIllegalPositions() throws Exception {
-    Directory dir = newDirectory();
-    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
-    Document doc = new Document();
-    Token t1 = new Token("foo", 0, 3);
-    t1.setPositionIncrement(Integer.MAX_VALUE);
-    Token t2 = new Token("bar", 4, 7);
-    t2.setPositionIncrement(200);
-    TokenStream overflowingTokenStream = new CannedTokenStream(
-        new Token[] { t1, t2 }
-    );
-    Field field = new TextField("foo", overflowingTokenStream);
-    doc.add(field);
-    try {
-      iw.addDocument(doc);
-      fail();
-    } catch (IllegalArgumentException expected) {
-      // expected exception
-    }
-    iw.close();
-    dir.close();
-  }
-  
-  public void testLegalbutVeryLargePositions() throws Exception {
-    Directory dir = newDirectory();
-    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
-    Document doc = new Document();
-    Token t1 = new Token("foo", 0, 3);
-    t1.setPositionIncrement(Integer.MAX_VALUE-500);
-    if (random().nextBoolean()) {
-      t1.setPayload(new BytesRef(new byte[] { 0x1 } ));
-    }
-    TokenStream overflowingTokenStream = new CannedTokenStream(
-        new Token[] { t1 }
-    );
-    Field field = new TextField("foo", overflowingTokenStream);
-    doc.add(field);
-    iw.addDocument(doc);
-    iw.close();
-    dir.close();
-  }
-  
+
   public void testBoostOmitNorms() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/util/automaton/TestAutomaton.java Tue Apr 14 19:12:09 2015
@@ -1105,34 +1105,154 @@ public class TestAutomaton extends Lucen
     }
   }
 
-  public void testMakeBinaryIntervalRandom() throws Exception {
+  private boolean accepts(Automaton a, BytesRef b) {
+    IntsRefBuilder intsBuilder = new IntsRefBuilder();
+    Util.toIntsRef(b, intsBuilder);    
+    return Operations.run(a, intsBuilder.toIntsRef());
+  }
+
+  private Automaton makeBinaryInterval(BytesRef minTerm, boolean minInclusive,
+                                       BytesRef maxTerm, boolean maxInclusive) {
+    
+    if (VERBOSE) {
+      System.out.println("TEST: minTerm=" + minTerm + " minInclusive=" + minInclusive + " maxTerm=" + maxTerm + " maxInclusive=" + maxInclusive);
+    }
+
+    Automaton a = Automata.makeBinaryInterval(minTerm, minInclusive,
+                                              maxTerm, maxInclusive);
+
+    Automaton minA = MinimizationOperations.minimize(a, Integer.MAX_VALUE);
+    if (minA.getNumStates() != a.getNumStates()) {
+      assertTrue(minA.getNumStates() < a.getNumStates());
+      System.out.println("Original was not minimal:");
+      System.out.println("Original:\n" + a.toDot());
+      System.out.println("Minimized:\n" + minA.toDot());
+      System.out.println("minTerm=" + minTerm + " minInclusive=" + minInclusive);
+      System.out.println("maxTerm=" + maxTerm + " maxInclusive=" + maxInclusive);
+      fail("automaton was not minimal");
+    }
+
+    if (VERBOSE) {
+      System.out.println(a.toDot());
+    }
+
+    return a;
+  }
+
+  public void testMakeBinaryIntervalFiniteCasesBasic() throws Exception {
+    // 0 (incl) - 00 (incl)
+    byte[] zeros = new byte[3];
+    Automaton a = makeBinaryInterval(new BytesRef(zeros, 0, 1), true, new BytesRef(zeros, 0, 2), true);
+    assertTrue(Operations.isFinite(a));
+    assertFalse(accepts(a, new BytesRef()));
+    assertTrue(accepts(a, new BytesRef(zeros, 0, 1)));
+    assertTrue(accepts(a, new BytesRef(zeros, 0, 2)));
+    assertFalse(accepts(a, new BytesRef(zeros, 0, 3)));
+
+    // '' (incl) - 00 (incl)
+    a = makeBinaryInterval(new BytesRef(), true, new BytesRef(zeros, 0, 2), true);
+    assertTrue(Operations.isFinite(a));
+    assertTrue(accepts(a, new BytesRef()));
+    assertTrue(accepts(a, new BytesRef(zeros, 0, 1)));
+    assertTrue(accepts(a, new BytesRef(zeros, 0, 2)));
+    assertFalse(accepts(a, new BytesRef(zeros, 0, 3)));
+
+    // '' (excl) - 00 (incl)
+    a = makeBinaryInterval(new BytesRef(), false, new BytesRef(zeros, 0, 2), true);
+    assertTrue(Operations.isFinite(a));
+    assertFalse(accepts(a, new BytesRef()));
+    assertTrue(accepts(a, new BytesRef(zeros, 0, 1)));
+    assertTrue(accepts(a, new BytesRef(zeros, 0, 2)));
+    assertFalse(accepts(a, new BytesRef(zeros, 0, 3)));
+
+    // 0 (excl) - 00 (incl)
+    a = makeBinaryInterval(new BytesRef(zeros, 0, 1), false, new BytesRef(zeros, 0, 2), true);
+    assertTrue(Operations.isFinite(a));
+    assertFalse(accepts(a, new BytesRef()));
+    assertFalse(accepts(a, new BytesRef(zeros, 0, 1)));
+    assertTrue(accepts(a, new BytesRef(zeros, 0, 2)));
+    assertFalse(accepts(a, new BytesRef(zeros, 0, 3)));
+
+    // 0 (excl) - 00 (excl)
+    a = makeBinaryInterval(new BytesRef(zeros, 0, 1), false, new BytesRef(zeros, 0, 2), false);
+    assertTrue(Operations.isFinite(a));
+    assertFalse(accepts(a, new BytesRef()));
+    assertFalse(accepts(a, new BytesRef(zeros, 0, 1)));
+    assertFalse(accepts(a, new BytesRef(zeros, 0, 2)));
+    assertFalse(accepts(a, new BytesRef(zeros, 0, 3)));
+  }
+
+  public void testMakeBinaryIntervalFiniteCasesRandom() throws Exception {
     int iters = atLeast(100);
     for(int iter=0;iter<iters;iter++) {
-      BytesRef minTerm = TestUtil.randomBinaryTerm(random());
+      BytesRef prefix = new BytesRef(TestUtil.randomRealisticUnicodeString(random()));
+
+      BytesRefBuilder b = new BytesRefBuilder();
+      b.append(prefix);
+      int numZeros = random().nextInt(10);
+      for(int i=0;i<numZeros;i++) {
+        b.append((byte) 0);
+      }
+      BytesRef minTerm = b.get();
+
+      b = new BytesRefBuilder();
+      b.append(minTerm);
+      numZeros = random().nextInt(10);
+      for(int i=0;i<numZeros;i++) {
+        b.append((byte) 0);
+      }
+      BytesRef maxTerm = b.get();
+      
       boolean minInclusive = random().nextBoolean();
-      BytesRef maxTerm = TestUtil.randomBinaryTerm(random());
       boolean maxInclusive = random().nextBoolean();
-
-      if (VERBOSE) {
-        System.out.println("TEST: iter=" + iter + " minTerm=" + minTerm + " minInclusive=" + minInclusive + " maxTerm=" + maxTerm + " maxInclusive=" + maxInclusive);
+      Automaton a = makeBinaryInterval(minTerm, minInclusive,
+                                       maxTerm, maxInclusive);
+      assertTrue(Operations.isFinite(a));
+      int expectedCount = maxTerm.length - minTerm.length + 1;
+      if (minInclusive == false) {
+        expectedCount--;
+      }
+      if (maxInclusive == false) {
+        expectedCount--;
       }
 
-      Automaton a = Automata.makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive);
+      if (expectedCount <= 0) {
+        assertTrue(Operations.isEmpty(a));
+        continue;
+      } else {
+        // Enumerate all finite strings and verify the count matches what we expect:
+        assertEquals(expectedCount, Operations.getFiniteStrings(a, expectedCount).size());
+      }
 
-      Automaton minA = MinimizationOperations.minimize(a, Integer.MAX_VALUE);
-      if (minA.getNumStates() != a.getNumStates()) {
-        assertTrue(minA.getNumStates() < a.getNumStates());
-        System.out.println("Original was not minimal:");
-        System.out.println("Original:\n" + a.toDot());
-        System.out.println("Minimized:\n" + minA.toDot());
-        System.out.println("minTerm=" + minTerm + " minInclusive=" + minInclusive);
-        System.out.println("maxTerm=" + maxTerm + " maxInclusive=" + maxInclusive);
-        fail("automaton was not minimal");
+      b = new BytesRefBuilder();
+      b.append(minTerm);
+      if (minInclusive == false) {
+        assertFalse(accepts(a, b.get()));
+        b.append((byte) 0);
       }
+      while (b.length() < maxTerm.length) {
+        b.append((byte) 0);
 
-      if (VERBOSE) {
-        System.out.println(a.toDot());
+        boolean expected;
+        if (b.length() == maxTerm.length) {
+          expected = maxInclusive;
+        } else {
+          expected = true;
+        }
+        assertEquals(expected, accepts(a, b.get()));
       }
+    }
+  }
+
+  public void testMakeBinaryIntervalRandom() throws Exception {
+    int iters = atLeast(100);
+    for(int iter=0;iter<iters;iter++) {
+      BytesRef minTerm = TestUtil.randomBinaryTerm(random());
+      boolean minInclusive = random().nextBoolean();
+      BytesRef maxTerm = TestUtil.randomBinaryTerm(random());
+      boolean maxInclusive = random().nextBoolean();
+
+      Automaton a = makeBinaryInterval(minTerm, minInclusive, maxTerm, maxInclusive);
 
       for(int iter2=0;iter2<500;iter2++) {
         BytesRef term = TestUtil.randomBinaryTerm(random());

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java?rev=1673526&r1=1673525&r2=1673526&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java Tue Apr 14 19:12:09 2015
@@ -25,10 +25,11 @@ import org.apache.lucene.codecs.FieldsCo
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.index.AssertingLeafReader;
-import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
@@ -219,6 +220,7 @@ public final class AssertingPostingsForm
                 for(int i=0;i<freq;i++) {
                   int pos = postingsEnum.nextPosition();
                   assert pos >= lastPos: "pos=" + pos + " vs lastPos=" + lastPos + " i=" + i + " freq=" + freq;
+                  assert pos <= IndexWriter.MAX_POSITION: "pos=" + pos + " is > IndexWriter.MAX_POSITION=" + IndexWriter.MAX_POSITION;
                   lastPos = pos;
 
                   if (hasOffsets) {