You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2015/04/02 17:37:41 UTC

svn commit: r1670929 [2/5] - in /lucene/dev/branches/lucene6271: ./ lucene/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/autoprefix/ lucene/codecs/src/resources/META-INF/services/ lucene/codecs/src/test/org/apache/lucene/codecs/autopr...

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java Thu Apr  2 15:37:39 2015
@@ -35,9 +35,14 @@ final class IntersectTermsEnumFrame {
   long fpEnd;
   long lastSubFP;
 
+  // private static boolean DEBUG = IntersectTermsEnum.DEBUG;
+
   // State in automaton
   int state;
 
+  // State just before the last label
+  int lastState;
+
   int metaDataUpto;
 
   byte[] suffixBytes = new byte[128];
@@ -73,6 +78,8 @@ final class IntersectTermsEnumFrame {
   int transitionIndex;
   int transitionCount;
 
+  final boolean versionAutoPrefix;
+
   FST.Arc<BytesRef> arc;
 
   final BlockTermState termState;
@@ -89,6 +96,17 @@ final class IntersectTermsEnumFrame {
   int startBytePos;
   int suffix;
 
+  // When we are on an auto-prefix term this is the starting lead byte
+  // of the suffix (e.g. 'a' for the foo[a-m]* case):
+  int floorSuffixLeadStart;
+
+  // When we are on an auto-prefix term this is the ending lead byte
+  // of the suffix (e.g. 'm' for the foo[a-m]* case):
+  int floorSuffixLeadEnd;
+
+  // True if the term we are currently on is an auto-prefix term:
+  boolean isAutoPrefixTerm;
+
   private final IntersectTermsEnum ite;
 
   public IntersectTermsEnumFrame(IntersectTermsEnum ite, int ord) throws IOException {
@@ -97,35 +115,39 @@ final class IntersectTermsEnumFrame {
     this.termState = ite.fr.parent.postingsReader.newTermState();
     this.termState.totalTermFreq = -1;
     this.longs = new long[ite.fr.longsSize];
+    this.versionAutoPrefix = ite.fr.parent.version >= BlockTreeTermsReader.VERSION_AUTO_PREFIX_TERMS;
   }
 
   void loadNextFloorBlock() throws IOException {
     assert numFollowFloorBlocks > 0;
-    //if (DEBUG) System.out.println("    loadNextFoorBlock trans=" + transitions[transitionIndex]);
+    //if (DEBUG) System.out.println("    loadNextFloorBlock transition.min=" + transition.min);
 
     do {
       fp = fpOrig + (floorDataReader.readVLong() >>> 1);
       numFollowFloorBlocks--;
-      // if (DEBUG) System.out.println("    skip floor block2!  nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[transitionIndex].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+      //if (DEBUG) System.out.println("    skip floor block2!  nextFloorLabel=" + (char) nextFloorLabel + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
       if (numFollowFloorBlocks != 0) {
         nextFloorLabel = floorDataReader.readByte() & 0xff;
       } else {
         nextFloorLabel = 256;
       }
-      // if (DEBUG) System.out.println("    nextFloorLabel=" + (char) nextFloorLabel);
+      //if (DEBUG) System.out.println("    nextFloorLabel=" + (char) nextFloorLabel);
     } while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min);
 
+    //if (DEBUG) System.out.println("      done loadNextFloorBlock");
+
     load(null);
   }
 
   public void setState(int state) {
     this.state = state;
     transitionIndex = 0;
-    transitionCount = ite.compiledAutomaton.automaton.getNumTransitions(state);
+    transitionCount = ite.automaton.getNumTransitions(state);
     if (transitionCount != 0) {
-      ite.compiledAutomaton.automaton.initTransition(state, transition);
-      ite.compiledAutomaton.automaton.getNextTransition(transition);
+      ite.automaton.initTransition(state, transition);
+      ite.automaton.getNextTransition(transition);
       curTransitionMax = transition.max;
+      //if (DEBUG) System.out.println("    after setState state=" + state + " trans: " + transition + " transCount=" + transitionCount);
     } else {
       curTransitionMax = -1;
     }
@@ -133,7 +155,7 @@ final class IntersectTermsEnumFrame {
 
   void load(BytesRef frameIndexData) throws IOException {
 
-    // if (DEBUG) System.out.println("    load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state));
+    //xif (DEBUG) System.out.println("    load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state));
 
     if (frameIndexData != null && transitionCount != 0) {
       // Floor frame
@@ -148,7 +170,7 @@ final class IntersectTermsEnumFrame {
       if ((code & BlockTreeTermsReader.OUTPUT_FLAG_IS_FLOOR) != 0) {
         numFollowFloorBlocks = floorDataReader.readVInt();
         nextFloorLabel = floorDataReader.readByte() & 0xff;
-        // if (DEBUG) System.out.println("    numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
+        //if (DEBUG) System.out.println("    numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
 
         // If current state is accept, we must process
         // first block in case it has empty suffix:
@@ -158,7 +180,7 @@ final class IntersectTermsEnumFrame {
           while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min) {
             fp = fpOrig + (floorDataReader.readVLong() >>> 1);
             numFollowFloorBlocks--;
-            // if (DEBUG) System.out.println("    skip floor block!  nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+            //xif (DEBUG) System.out.println("    skip floor block!  nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
             if (numFollowFloorBlocks != 0) {
               nextFloorLabel = floorDataReader.readByte() & 0xff;
             } else {
@@ -179,7 +201,7 @@ final class IntersectTermsEnumFrame {
     code = ite.in.readVInt();
     isLeafBlock = (code & 1) != 0;
     int numBytes = code >>> 1;
-    // if (DEBUG) System.out.println("      entCount=" + entCount + " lastInFloor?=" + isLastInFloor + " leafBlock?=" + isLeafBlock + " numSuffixBytes=" + numBytes);
+    //if (DEBUG) System.out.println("      entCount=" + entCount + " lastInFloor?=" + isLastInFloor + " leafBlock?=" + isLeafBlock + " numSuffixBytes=" + numBytes);
     if (suffixBytes.length < numBytes) {
       suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
     }
@@ -214,41 +236,106 @@ final class IntersectTermsEnumFrame {
       // written one after another -- tail recurse:
       fpEnd = ite.in.getFilePointer();
     }
+
+    // Necessary in case this ord previously was an auto-prefix
+    // term but now we recurse to a new leaf block
+    isAutoPrefixTerm = false;
   }
 
   // TODO: maybe add scanToLabel; should give perf boost
 
+  // Decodes next entry; returns true if it's a sub-block
   public boolean next() {
-    return isLeafBlock ? nextLeaf() : nextNonLeaf();
+    if (isLeafBlock) {
+      nextLeaf();
+      return false;
+    } else {
+      return nextNonLeaf();
+    }
   }
 
-  // Decodes next entry; returns true if it's a sub-block
-  public boolean nextLeaf() {
-    //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+  public void nextLeaf() {
+    //if (DEBUG) {
+    //  System.out.println("  frame.nextLeaf ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+    //}
     assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
     nextEnt++;
     suffix = suffixesReader.readVInt();
     startBytePos = suffixesReader.getPosition();
     suffixesReader.skipBytes(suffix);
-    return false;
   }
 
   public boolean nextNonLeaf() {
-    //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+    //if (DEBUG) {
+    //  System.out.println("  frame.nextNonLeaf ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount + " versionAutoPrefix=" + versionAutoPrefix + " fp=" + suffixesReader.getPosition());
+    // }
     assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
     nextEnt++;
     final int code = suffixesReader.readVInt();
-    suffix = code >>> 1;
-    startBytePos = suffixesReader.getPosition();
-    suffixesReader.skipBytes(suffix);
-    if ((code & 1) == 0) {
-      // A normal term
-      termState.termBlockOrd++;
-      return false;
+    if (versionAutoPrefix == false) {
+      suffix = code >>> 1;
+      startBytePos = suffixesReader.getPosition();
+      suffixesReader.skipBytes(suffix);
+      if ((code & 1) == 0) {
+        // A normal term
+        termState.termBlockOrd++;
+        return false;
+      } else {
+        // A sub-block; make sub-FP absolute:
+        lastSubFP = fp - suffixesReader.readVLong();
+        return true;
+      }
     } else {
-      // A sub-block; make sub-FP absolute:
-      lastSubFP = fp - suffixesReader.readVLong();
-      return true;
+      suffix = code >>> 2;
+      startBytePos = suffixesReader.getPosition();
+      suffixesReader.skipBytes(suffix);
+      switch (code & 3) {
+      case 0:
+        // A normal term
+        //if (DEBUG) System.out.println("    ret: term");
+        isAutoPrefixTerm = false;
+        termState.termBlockOrd++;
+        return false;
+      case 1:
+        // A sub-block; make sub-FP absolute:
+        isAutoPrefixTerm = false;
+        lastSubFP = fp - suffixesReader.readVLong();
+        //if (DEBUG) System.out.println("    ret: sub-block");
+        return true;
+      case 2:
+        // A normal prefix term, suffix leads with empty string
+        floorSuffixLeadStart = -1;
+        termState.termBlockOrd++;
+        floorSuffixLeadEnd = suffixesReader.readByte() & 0xff;
+        if (floorSuffixLeadEnd == 0xff) {
+          floorSuffixLeadEnd = -1;
+          //System.out.println("  fill in -1");
+        }
+        //if (DEBUG) System.out.println("    ret: floor prefix term: start=-1 end=" + floorSuffixLeadEnd);
+        isAutoPrefixTerm = true;
+        return false;
+      case 3:
+        // A floor'd prefix term, suffix leads with real byte
+        if (suffix == 0) {
+          // TODO: this is messy, but necessary because we are an auto-prefix term, but our suffix is the empty string here, so we have to
+          // look at the parent block to get the lead suffix byte:
+          assert ord > 0;
+          IntersectTermsEnumFrame parent = ite.stack[ord-1];
+          floorSuffixLeadStart = parent.suffixBytes[parent.startBytePos+parent.suffix-1] & 0xff;
+          //if (DEBUG) System.out.println("    peek-parent: suffix=" + floorSuffixLeadStart);
+        } else {
+          floorSuffixLeadStart = suffixBytes[startBytePos+suffix-1] & 0xff;
+        }
+        termState.termBlockOrd++;
+        isAutoPrefixTerm = true;
+        floorSuffixLeadEnd = suffixesReader.readByte() & 0xff;
+        //if (DEBUG) System.out.println("    ret: floor prefix term start=" + floorSuffixLeadStart + " end=" + floorSuffixLeadEnd);
+        return false;
+      default:
+        // Silly javac:
+        assert false;
+        return false;
+      }
     }
   }
 

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java Thu Apr  2 15:37:39 2015
@@ -34,7 +34,9 @@ import org.apache.lucene.util.RamUsageEs
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.Util;
 
-/** Iterates through terms in this field */
+/** Iterates through terms in this field.  This implementation skips
+ *  any auto-prefix terms it encounters. */
+
 final class SegmentTermsEnum extends TermsEnum {
 
   // Lazy init:
@@ -48,7 +50,7 @@ final class SegmentTermsEnum extends Ter
 
   private int targetBeforeCurrentLength;
 
-  // static boolean DEBUG = false;
+  //static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
 
   private final ByteArrayDataInput scratchReader = new ByteArrayDataInput();
 
@@ -119,6 +121,8 @@ final class SegmentTermsEnum extends Ter
    *  computing aggregate statistics. */
   public Stats computeBlockStats() throws IOException {
 
+    // TODO: add total auto-prefix term count
+
     Stats stats = new Stats(fr.parent.segment, fr.fieldInfo.name);
     if (fr.index != null) {
       stats.indexNodeCount = fr.index.getNodeCount();
@@ -152,8 +156,10 @@ final class SegmentTermsEnum extends Ter
       while (currentFrame.nextEnt == currentFrame.entCount) {
         stats.endBlock(currentFrame);
         if (!currentFrame.isLastInFloor) {
+          // Advance to next floor block
           currentFrame.loadNextFloorBlock();
           stats.startBlock(currentFrame, true);
+          break;
         } else {
           if (currentFrame.ord == 0) {
             break allTerms;
@@ -175,8 +181,6 @@ final class SegmentTermsEnum extends Ter
           // This is a "next" frame -- even if it's
           // floor'd we must pretend it isn't so we don't
           // try to scan to the right floor frame:
-          currentFrame.isFloor = false;
-          //currentFrame.hasTerms = true;
           currentFrame.loadBlock();
           stats.startBlock(currentFrame, !currentFrame.isLastInFloor);
         } else {
@@ -294,6 +298,7 @@ final class SegmentTermsEnum extends Ter
     return true;
   }
 
+  /*
   // for debugging
   @SuppressWarnings("unused")
   static String brToString(BytesRef b) {
@@ -307,8 +312,15 @@ final class SegmentTermsEnum extends Ter
     }
   }
 
+  // for debugging
+  @SuppressWarnings("unused")
+  static String brToString(BytesRefBuilder b) {
+    return brToString(b.get());
+  }
+  */
+
   @Override
-  public boolean seekExact(final BytesRef target) throws IOException {
+  public boolean seekExact(BytesRef target) throws IOException {
 
     if (fr.index == null) {
       throw new IllegalStateException("terms index was not loaded");
@@ -565,7 +577,8 @@ final class SegmentTermsEnum extends Ter
   }
 
   @Override
-  public SeekStatus seekCeil(final BytesRef target) throws IOException {
+  public SeekStatus seekCeil(BytesRef target) throws IOException {
+
     if (fr.index == null) {
       throw new IllegalStateException("terms index was not loaded");
     }
@@ -575,7 +588,7 @@ final class SegmentTermsEnum extends Ter
     assert clearEOF();
 
     // if (DEBUG) {
-    //   System.out.println("\nBTTR.seekCeil seg=" + fr.parent.segment + " target=" + fr.fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix=  " + validIndexPrefix);
+    //   System.out.println("\nBTTR.seekCeil seg=" + fr.parent.segment + " target=" + fr.fieldInfo.name + ":" + brToString(target) + " " + target + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix=  " + validIndexPrefix);
     //   printSeekState(System.out);
     // }
 
@@ -617,7 +630,7 @@ final class SegmentTermsEnum extends Ter
       while (targetUpto < targetLimit) {
         cmp = (term.byteAt(targetUpto)&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
         //if (DEBUG) {
-        //System.out.println("    cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")"   + " arc.output=" + arc.output + " output=" + output);
+        //System.out.println("    cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.byteAt(targetUpto)) + ")"   + " arc.output=" + arc.output + " output=" + output);
         //}
         if (cmp != 0) {
           break;
@@ -647,7 +660,7 @@ final class SegmentTermsEnum extends Ter
         while (targetUpto < targetLimit2) {
           cmp = (term.byteAt(targetUpto)&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
           //if (DEBUG) {
-          //System.out.println("    cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")");
+          //System.out.println("    cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.byteAt(targetUpto)) + ")");
           //}
           if (cmp != 0) {
             break;
@@ -733,7 +746,7 @@ final class SegmentTermsEnum extends Ter
 
         // Index is exhausted
         // if (DEBUG) {
-        //   System.out.println("    index: index exhausted label=" + ((char) targetLabel) + " " + toHex(targetLabel));
+        //   System.out.println("    index: index exhausted label=" + ((char) targetLabel) + " " + targetLabel);
         // }
             
         validIndexPrefix = currentFrame.prefix;
@@ -743,6 +756,7 @@ final class SegmentTermsEnum extends Ter
 
         currentFrame.loadBlock();
 
+        //if (DEBUG) System.out.println("  now scanToTerm");
         final SeekStatus result = currentFrame.scanToTerm(target, false);
         if (result == SeekStatus.END) {
           term.copyBytes(target);
@@ -750,7 +764,7 @@ final class SegmentTermsEnum extends Ter
 
           if (next() != null) {
             //if (DEBUG) {
-            //System.out.println("  return NOT_FOUND term=" + brToString(term) + " " + term);
+            //System.out.println("  return NOT_FOUND term=" + brToString(term));
             //}
             return SeekStatus.NOT_FOUND;
           } else {
@@ -761,7 +775,7 @@ final class SegmentTermsEnum extends Ter
           }
         } else {
           //if (DEBUG) {
-          //System.out.println("  return " + result + " term=" + brToString(term) + " " + term);
+          //System.out.println("  return " + result + " term=" + brToString(term));
           //}
           return result;
         }
@@ -776,7 +790,7 @@ final class SegmentTermsEnum extends Ter
         }
 
         //if (DEBUG) {
-        //System.out.println("    index: follow label=" + toHex(target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
+        //System.out.println("    index: follow label=" + (target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
         //}
         targetUpto++;
 
@@ -802,7 +816,7 @@ final class SegmentTermsEnum extends Ter
       termExists = false;
       if (next() != null) {
         //if (DEBUG) {
-        //System.out.println("  return NOT_FOUND term=" + term.utf8ToString() + " " + term);
+        //System.out.println("  return NOT_FOUND term=" + term.get().utf8ToString() + " " + term);
         //}
         return SeekStatus.NOT_FOUND;
       } else {
@@ -906,7 +920,9 @@ final class SegmentTermsEnum extends Ter
     // Pop finished blocks
     while (currentFrame.nextEnt == currentFrame.entCount) {
       if (!currentFrame.isLastInFloor) {
+        // Advance to next floor block
         currentFrame.loadNextFloorBlock();
+        break;
       } else {
         //if (DEBUG) System.out.println("  pop frame");
         if (currentFrame.ord == 0) {
@@ -946,11 +962,9 @@ final class SegmentTermsEnum extends Ter
         // This is a "next" frame -- even if it's
         // floor'd we must pretend it isn't so we don't
         // try to scan to the right floor frame:
-        currentFrame.isFloor = false;
-        //currentFrame.hasTerms = true;
         currentFrame.loadBlock();
       } else {
-        //if (DEBUG) System.out.println("  return term=" + term.utf8ToString() + " " + term + " currentFrame.ord=" + currentFrame.ord);
+        //if (DEBUG) System.out.println("  return term=" + brToString(term) + " currentFrame.ord=" + currentFrame.ord);
         return term.get();
       }
     }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java Thu Apr  2 15:37:39 2015
@@ -37,6 +37,10 @@ final class SegmentTermsEnumFrame {
 
   FST.Arc<BytesRef> arc;
 
+  final boolean versionAutoPrefix;
+
+  //static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
   // File pointer where this block was loaded from
   long fp;
   long fpOrig;
@@ -96,6 +100,7 @@ final class SegmentTermsEnumFrame {
     this.state = ste.fr.parent.postingsReader.newTermState();
     this.state.totalTermFreq = -1;
     this.longs = new long[ste.fr.longsSize];
+    this.versionAutoPrefix = ste.fr.parent.version >= BlockTreeTermsReader.VERSION_AUTO_PREFIX_TERMS;
   }
 
   public void setFloorData(ByteArrayDataInput in, BytesRef source) {
@@ -262,12 +267,17 @@ final class SegmentTermsEnumFrame {
     */
   }
 
-  public boolean next() {
-    return isLeafBlock ? nextLeaf() : nextNonLeaf();
+  // Decodes next entry; returns true if it's a sub-block
+  public boolean next() throws IOException {
+    if (isLeafBlock) {
+      nextLeaf();
+      return false;
+    } else {
+      return nextNonLeaf();
+    }
   }
 
-  // Decodes next entry; returns true if it's a sub-block
-  public boolean nextLeaf() {
+  public void nextLeaf() {
     //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
     assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
     nextEnt++;
@@ -276,36 +286,78 @@ final class SegmentTermsEnumFrame {
     ste.term.setLength(prefix + suffix);
     ste.term.grow(ste.term.length());
     suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
-    // A normal term
     ste.termExists = true;
-    return false;
   }
 
-  public boolean nextNonLeaf() {
-    //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
-    assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
-    nextEnt++;
-    final int code = suffixesReader.readVInt();
-    suffix = code >>> 1;
-    startBytePos = suffixesReader.getPosition();
-    ste.term.setLength(prefix + suffix);
-    ste.term.grow(ste.term.length());
-    suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
-    if ((code & 1) == 0) {
-      // A normal term
-      ste.termExists = true;
-      subCode = 0;
-      state.termBlockOrd++;
-      return false;
-    } else {
-      // A sub-block; make sub-FP absolute:
-      ste.termExists = false;
-      subCode = suffixesReader.readVLong();
-      lastSubFP = fp - subCode;
-      //if (DEBUG) {
-      //System.out.println("    lastSubFP=" + lastSubFP);
-      //}
-      return true;
+  public boolean nextNonLeaf() throws IOException {
+    //if (DEBUG) System.out.println("  stef.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + suffixesReader.getPosition());
+    while (true) {
+      if (nextEnt == entCount) {
+        assert arc == null || (isFloor && isLastInFloor == false): "isFloor=" + isFloor + " isLastInFloor=" + isLastInFloor;
+        loadNextFloorBlock();
+        if (isLeafBlock) {
+          nextLeaf();
+          return false;
+        } else {
+          continue;
+        }
+      }
+        
+      assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+      nextEnt++;
+      final int code = suffixesReader.readVInt();
+      if (versionAutoPrefix == false) {
+        suffix = code >>> 1;
+      } else {
+        suffix = code >>> 2;
+      }
+      startBytePos = suffixesReader.getPosition();
+      ste.term.setLength(prefix + suffix);
+      ste.term.grow(ste.term.length());
+      suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
+      if (versionAutoPrefix == false) {
+        if ((code & 1) == 0) {
+          // A normal term
+          ste.termExists = true;
+          subCode = 0;
+          state.termBlockOrd++;
+          return false;
+        } else {
+          // A sub-block; make sub-FP absolute:
+          ste.termExists = false;
+          subCode = suffixesReader.readVLong();
+          lastSubFP = fp - subCode;
+          //if (DEBUG) {
+          //System.out.println("    lastSubFP=" + lastSubFP);
+          //}
+          return true;
+        }
+      } else {
+
+        switch(code & 3) {
+        case 0:
+          // A normal term
+          ste.termExists = true;
+          subCode = 0;
+          state.termBlockOrd++;
+          return false;
+        case 1:
+          // A sub-block; make sub-FP absolute:
+          ste.termExists = false;
+          subCode = suffixesReader.readVLong();
+          lastSubFP = fp - subCode;
+          //if (DEBUG) {
+          //System.out.println("    lastSubFP=" + lastSubFP);
+          //}
+          return true;
+        case 2:
+        case 3:
+          // A prefix term: skip it
+          state.termBlockOrd++;
+          suffixesReader.readByte();
+          continue;
+        }
+      }
     }
   }
         
@@ -448,18 +500,38 @@ final class SegmentTermsEnumFrame {
       assert nextEnt < entCount;
       nextEnt++;
       final int code = suffixesReader.readVInt();
-      suffixesReader.skipBytes(isLeafBlock ? code : code >>> 1);
-      //if (DEBUG) System.out.println("    " + nextEnt + " (of " + entCount + ") ent isSubBlock=" + ((code&1)==1));
-      if ((code & 1) != 0) {
-        final long subCode = suffixesReader.readVLong();
-        //if (DEBUG) System.out.println("      subCode=" + subCode);
-        if (targetSubCode == subCode) {
-          //if (DEBUG) System.out.println("        match!");
-          lastSubFP = subFP;
-          return;
+      if (versionAutoPrefix == false) {
+        suffixesReader.skipBytes(code >>> 1);
+        if ((code & 1) != 0) {
+          final long subCode = suffixesReader.readVLong();
+          if (targetSubCode == subCode) {
+            //if (DEBUG) System.out.println("        match!");
+            lastSubFP = subFP;
+            return;
+          }
+        } else {
+          state.termBlockOrd++;
         }
       } else {
-        state.termBlockOrd++;
+        int flag = code & 3;
+        suffixesReader.skipBytes(code >>> 2);
+        //if (DEBUG) System.out.println("    " + nextEnt + " (of " + entCount + ") ent isSubBlock=" + ((code&1)==1));
+        if (flag == 1) {
+          // Sub-block
+          final long subCode = suffixesReader.readVLong();
+          //if (DEBUG) System.out.println("      subCode=" + subCode);
+          if (targetSubCode == subCode) {
+            //if (DEBUG) System.out.println("        match!");
+            lastSubFP = subFP;
+            return;
+          }
+        } else {
+          state.termBlockOrd++;
+          if (flag == 2 || flag == 3) {
+            // Floor'd prefix term
+            suffixesReader.readByte();
+          }
+        }
       }
     }
   }
@@ -473,6 +545,21 @@ final class SegmentTermsEnumFrame {
   private int suffix;
   private long subCode;
 
+  // for debugging
+  /*
+  @SuppressWarnings("unused")
+  static String brToString(BytesRef b) {
+    try {
+      return b.utf8ToString() + " " + b;
+    } catch (Throwable t) {
+      // If BytesRef isn't actually UTF8, or it's eg a
+      // prefix of UTF8 that ends mid-unicode-char, we
+      // fallback to hex:
+      return b.toString();
+    }
+  }
+  */
+
   // Target's prefix matches this block's prefix; we
   // scan the entries check if the suffix matches.
   public SeekStatus scanToTermLeaf(BytesRef target, boolean exactOnly) throws IOException {
@@ -535,9 +622,6 @@ final class SegmentTermsEnumFrame {
           // keep scanning
 
           if (nextEnt == entCount) {
-            if (exactOnly) {
-              fillTerm();
-            }
             // We are done scanning this block
             break nextTerm;
           } else {
@@ -590,7 +674,7 @@ final class SegmentTermsEnumFrame {
   // scan the entries check if the suffix matches.
   public SeekStatus scanToTermNonLeaf(BytesRef target, boolean exactOnly) throws IOException {
 
-    //if (DEBUG) System.out.println("    scanToTermNonLeaf: block fp=" + fp + " prefix=" + prefix + " nextEnt=" + nextEnt + " (of " + entCount + ") target=" + brToString(target) + " term=" + brToString(term));
+    //if (DEBUG) System.out.println("    scanToTermNonLeaf: block fp=" + fp + " prefix=" + prefix + " nextEnt=" + nextEnt + " (of " + entCount + ") target=" + brToString(target) + " term=" + brToString(target));
 
     assert nextEnt != -1;
 
@@ -605,30 +689,60 @@ final class SegmentTermsEnumFrame {
     assert prefixMatches(target);
 
     // Loop over each entry (term or sub-block) in this block:
-    //nextTerm: while(nextEnt < entCount) {
-    nextTerm: while (true) {
+    nextTerm: while(nextEnt < entCount) {
+
       nextEnt++;
 
       final int code = suffixesReader.readVInt();
-      suffix = code >>> 1;
-      // if (DEBUG) {
-      //   BytesRef suffixBytesRef = new BytesRef();
-      //   suffixBytesRef.bytes = suffixBytes;
-      //   suffixBytesRef.offset = suffixesReader.getPosition();
-      //   suffixBytesRef.length = suffix;
-      //   System.out.println("      cycle: " + ((code&1)==1 ? "sub-block" : "term") + " " + (nextEnt-1) + " (of " + entCount + ") suffix=" + brToString(suffixBytesRef));
-      // }
+      if (versionAutoPrefix == false) {
+        suffix = code >>> 1;
+      } else {
+        suffix = code >>> 2;
+      }
+
+      //if (DEBUG) {
+      //  BytesRef suffixBytesRef = new BytesRef();
+      //  suffixBytesRef.bytes = suffixBytes;
+      //  suffixBytesRef.offset = suffixesReader.getPosition();
+      //  suffixBytesRef.length = suffix;
+      //  System.out.println("      cycle: " + ((code&1)==1 ? "sub-block" : "term") + " " + (nextEnt-1) + " (of " + entCount + ") suffix=" + brToString(suffixBytesRef));
+      //}
 
-      ste.termExists = (code & 1) == 0;
       final int termLen = prefix + suffix;
       startBytePos = suffixesReader.getPosition();
       suffixesReader.skipBytes(suffix);
-      if (ste.termExists) {
-        state.termBlockOrd++;
-        subCode = 0;
+      if (versionAutoPrefix == false) {
+        ste.termExists = (code & 1) == 0;
+        if (ste.termExists) {
+          state.termBlockOrd++;
+          subCode = 0;
+        } else {
+          subCode = suffixesReader.readVLong();
+          lastSubFP = fp - subCode;
+        }
       } else {
-        subCode = suffixesReader.readVLong();
-        lastSubFP = fp - subCode;
+        switch (code & 3) {
+        case 0:
+          // Normal term
+          ste.termExists = true;
+          state.termBlockOrd++;
+          subCode = 0;
+          break;
+        case 1:
+          // Sub-block
+          ste.termExists = false;
+          subCode = suffixesReader.readVLong();
+          lastSubFP = fp - subCode;
+          break;
+        case 2:
+        case 3:
+          // Floor prefix term: skip it
+          //if (DEBUG) System.out.println("        skip floor prefix term");
+          suffixesReader.readByte();
+          ste.termExists = false;
+          state.termBlockOrd++;
+          continue;
+        }
       }
 
       final int targetLimit = target.offset + (target.length < termLen ? target.length : termLen);
@@ -637,7 +751,7 @@ final class SegmentTermsEnumFrame {
       // Loop over bytes in the suffix, comparing to
       // the target
       int bytePos = startBytePos;
-      while(true) {
+      while (true) {
         final int cmp;
         final boolean stop;
         if (targetPos < targetLimit) {
@@ -652,24 +766,18 @@ final class SegmentTermsEnumFrame {
         if (cmp < 0) {
           // Current entry is still before the target;
           // keep scanning
-
-          if (nextEnt == entCount) {
-            if (exactOnly) {
-              fillTerm();
-              //termExists = true;
-            }
-            // We are done scanning this block
-            break nextTerm;
-          } else {
-            continue nextTerm;
-          }
+          continue nextTerm;
         } else if (cmp > 0) {
 
           // Done!  Current entry is after target --
           // return NOT_FOUND:
           fillTerm();
 
+          //if (DEBUG) System.out.println("        maybe done exactOnly=" + exactOnly + " ste.termExists=" + ste.termExists);
+
           if (!exactOnly && !ste.termExists) {
+            //System.out.println("  now pushFrame");
+            // TODO this 
             // We are on a sub-block, and caller wants
             // us to position to the next term after
             // the target, so we must recurse into the

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java Thu Apr  2 15:37:39 2015
@@ -48,6 +48,8 @@ public class Stats {
   /** Total number of bytes (sum of term lengths) across all terms in the field. */
   public long totalTermBytes;
 
+  // TODO: add total auto-prefix term count
+
   /** The number of normal (non-floor) blocks in the terms file. */
   public int nonFloorBlockCount;
 

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java Thu Apr  2 15:37:39 2015
@@ -43,9 +43,9 @@ import org.apache.lucene.util.automaton.
  * completely accepted. This is not possible when the language accepted by the
  * FSM is not finite (i.e. * operator).
  * </p>
- * @lucene.experimental
+ * @lucene.internal
  */
-class AutomatonTermsEnum extends FilteredTermsEnum {
+public class AutomatonTermsEnum extends FilteredTermsEnum {
   // a tableized array-based form of the DFA
   private final ByteRunAutomaton runAutomaton;
   // common suffix of the automaton
@@ -70,9 +70,8 @@ class AutomatonTermsEnum extends Filtere
   /**
    * Construct an enumerator based upon an automaton, enumerating the specified
    * field, working on a supplied TermsEnum
-   * <p>
+   *
    * @lucene.experimental 
-   * <p>
    * @param compiled CompiledAutomaton
    */
   public AutomatonTermsEnum(TermsEnum tenum, CompiledAutomaton compiled) {

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Thu Apr  2 15:37:39 2015
@@ -25,7 +25,9 @@ import java.nio.file.Paths;
 import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Deque;
 import java.util.HashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -56,6 +58,8 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LongBitSet;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.Version;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 /**
  * Basic tool and API to check the health of an index and
@@ -902,6 +906,180 @@ public class CheckIndex implements Close
     return status;
   }
 
+  /** Visits all terms in the range minTerm (inclusive) to maxTerm (exclusive), marking all doc IDs encountered into allDocsSeen, and
+   *  returning the total number of terms visited. */
+  private static long getDocsFromTermRange(String field, int maxDoc, TermsEnum termsEnum, FixedBitSet docsSeen, BytesRef minTerm, BytesRef maxTerm, boolean isIntersect) throws IOException {
+    docsSeen.clear(0, docsSeen.length());
+
+    long termCount = 0;
+    PostingsEnum postingsEnum = null;
+    BytesRefBuilder lastTerm = null;
+    while (true) {
+      BytesRef term;
+
+      // Kinda messy: for intersect, we must first next(), but for "normal", we are already on our first term:
+      if (isIntersect || termCount != 0) {
+        term = termsEnum.next();
+      } else {
+        term = termsEnum.term();
+      }
+
+      if (term == null) {
+        if (isIntersect == false) {
+          throw new RuntimeException("didn't see max term field=" + field + " term=" + maxTerm);
+        }
+        return termCount;
+      }
+
+      assert term.isValid();
+        
+      if (lastTerm == null) {
+        lastTerm = new BytesRefBuilder();
+        lastTerm.copyBytes(term);
+      } else {
+        if (lastTerm.get().compareTo(term) >= 0) {
+          throw new RuntimeException("terms out of order: lastTerm=" + lastTerm + " term=" + term);
+        }
+        lastTerm.copyBytes(term);
+      }
+
+      //System.out.println("    term=" + term);
+
+      // Caller already ensured terms enum positioned >= minTerm:
+      if (term.compareTo(minTerm) < 0) {
+        throw new RuntimeException("saw term before min term field=" + field + " term=" + minTerm);
+      }
+
+      if (isIntersect == false) {
+        int cmp = term.compareTo(maxTerm);
+        if (cmp == 0) {
+          // Done!
+          return termCount;
+        } else if (cmp > 0) {
+          throw new RuntimeException("didn't see end term field=" + field + " term=" + maxTerm);
+        }
+      }
+
+      postingsEnum = termsEnum.postings(null, postingsEnum, 0);
+
+      int lastDoc = -1;
+      while (true) {
+        int doc = postingsEnum.nextDoc();
+        if (doc == DocIdSetIterator.NO_MORE_DOCS) {
+          break;
+        }
+        if (doc <= lastDoc) {
+          throw new RuntimeException("term " + term + ": doc " + doc + " <= lastDoc " + lastDoc);
+        }
+        if (doc >= maxDoc) {
+          throw new RuntimeException("term " + term + ": doc " + doc + " >= maxDoc " + maxDoc);
+        }
+
+        //System.out.println("      doc=" + doc);
+        docsSeen.set(doc);
+
+        lastDoc = doc;
+      }
+
+      termCount++;
+    }
+  }
+
+  /** Test Terms.intersect on this range, and validates that it returns the same doc ids as using non-intersect TermsEnum.  Returns true if
+   *  any fake terms were seen. */
+  private static boolean checkSingleTermRange(String field, int maxDoc, Terms terms, BytesRef minTerm, BytesRef maxTerm, FixedBitSet normalDocs, FixedBitSet intersectDocs) throws IOException {
+    // System.out.println("  check minTerm=" + minTerm + " maxTerm=" + maxTerm);
+
+    TermsEnum termsEnum = terms.iterator(null);
+    TermsEnum.SeekStatus status = termsEnum.seekCeil(minTerm);
+    if (status != TermsEnum.SeekStatus.FOUND) {
+      throw new RuntimeException("failed to seek to existing term field=" + field + " term=" + minTerm);
+    }
+
+    // Do "dumb" iteration to visit all terms in the range:
+    long normalTermCount = getDocsFromTermRange(field, maxDoc, termsEnum, normalDocs, minTerm, maxTerm, false);
+
+    // Now do the same operation using intersect:
+    long intersectTermCount = getDocsFromTermRange(field, maxDoc, terms.intersect(new CompiledAutomaton(Automata.makeBinaryInterval(minTerm, true, maxTerm, false), true, false, Integer.MAX_VALUE, true), null), intersectDocs, minTerm, maxTerm, true);
+
+    if (intersectTermCount > normalTermCount) {
+      throw new RuntimeException("intersect returned too many terms: field=" + field + " intersectTermCount=" + intersectTermCount + " normalTermCount=" + normalTermCount);
+    }
+
+    if (normalDocs.equals(intersectDocs) == false) {
+      throw new RuntimeException("intersect visited different docs than straight terms enum: " + normalDocs.cardinality() + " for straight enum, vs " + intersectDocs.cardinality() + " for intersect, minTerm=" + minTerm + " maxTerm=" + maxTerm);
+    }
+    //System.out.println("    " + intersectTermCount + " vs " + normalTermCount);
+    return intersectTermCount != normalTermCount;
+  }
+
+  /** Make an effort to visit "fake" (e.g. auto-prefix) terms.  We do this by running term range intersections across an initially wide
+   *  interval of terms, at different boundaries, and then gradually decrease the interval.  This is not guaranteed to hit all non-real
+   *  terms (doing that in general is non-trivial), but it should hit many of them, and validate their postings against the postings for the
+   *  real terms. */
+  private static void checkTermRanges(String field, int maxDoc, Terms terms, long numTerms) throws IOException {
+
+    // We'll target this many terms in our interval for the current level:
+    double currentInterval = numTerms;
+
+    FixedBitSet normalDocs = new FixedBitSet(maxDoc);
+    FixedBitSet intersectDocs = new FixedBitSet(maxDoc);
+
+    TermsEnum termsEnum = null;
+    //System.out.println("CI.checkTermRanges field=" + field + " numTerms=" + numTerms);
+
+    while (currentInterval >= 10.0) {
+      //System.out.println("  cycle interval=" + currentInterval);
+
+      // We iterate this terms enum to locate min/max term for each sliding/overlapping interval we test at the current level:
+      termsEnum = terms.iterator(termsEnum);
+
+      long termCount = 0;
+
+      Deque<BytesRef> termBounds = new LinkedList<>();
+
+      long lastTermAdded = Long.MIN_VALUE;
+
+      BytesRefBuilder lastTerm = null;
+
+      while (true) {
+        BytesRef term = termsEnum.next();
+        if (term == null) {
+          break;
+        }
+        //System.out.println("  top: term=" + term.utf8ToString());
+        if (termCount >= lastTermAdded + currentInterval/4) {
+          termBounds.add(BytesRef.deepCopyOf(term));
+          lastTermAdded = termCount;
+          if (termBounds.size() == 5) {
+            BytesRef minTerm = termBounds.removeFirst();
+            BytesRef maxTerm = termBounds.getLast();
+            checkSingleTermRange(field, maxDoc, terms, minTerm, maxTerm, normalDocs, intersectDocs);
+          }
+        }
+        termCount++;
+
+        if (lastTerm == null) {
+          lastTerm = new BytesRefBuilder();
+          lastTerm.copyBytes(term);
+        } else {
+          if (lastTerm.get().compareTo(term) >= 0) {
+            throw new RuntimeException("terms out of order: lastTerm=" + lastTerm + " term=" + term);
+          }
+          lastTerm.copyBytes(term);
+        }
+      }
+
+      if (lastTerm != null && termBounds.isEmpty() == false) {
+        BytesRef minTerm = termBounds.removeFirst();
+        BytesRef maxTerm = lastTerm.get();
+        checkSingleTermRange(field, maxDoc, terms, minTerm, maxTerm, normalDocs, intersectDocs);
+      }
+
+      currentInterval *= .75;
+    }
+  }
+
   /**
    * checks Fields api is consistent with itself.
    * searcher is optional, to verify with queries. Can be null.
@@ -922,6 +1100,7 @@ public class CheckIndex implements Close
     
     String lastField = null;
     for (String field : fields) {
+
       // MultiFieldsEnum relies upon this order...
       if (lastField != null && field.compareTo(lastField) <= 0) {
         throw new RuntimeException("fields out of order: lastField=" + lastField + " field=" + field);
@@ -1031,7 +1210,8 @@ public class CheckIndex implements Close
         if (term == null) {
           break;
         }
-
+        // System.out.println("CI: field=" + field + " check term=" + term + " docFreq=" + termsEnum.docFreq());
+        
         assert term.isValid();
         
         // make sure terms arrive in order according to
@@ -1323,13 +1503,21 @@ public class CheckIndex implements Close
         // docs got deleted and then merged away):
         
       } else {
+
+        long fieldTermCount = (status.delTermCount+status.termCount)-termCountStart;
+
+        if (hasFreqs == false) {
+          // For DOCS_ONLY fields we recursively test term ranges:
+          checkTermRanges(field, maxDoc, fieldTerms, fieldTermCount);
+        }
+
         final Object stats = fieldTerms.getStats();
         assert stats != null;
         if (status.blockTreeStats == null) {
           status.blockTreeStats = new HashMap<>();
         }
         status.blockTreeStats.put(field, stats);
-        
+
         if (sumTotalTermFreq != 0) {
           final long v = fields.terms(field).getSumTotalTermFreq();
           if (v != -1 && sumTotalTermFreq != v) {
@@ -1344,11 +1532,9 @@ public class CheckIndex implements Close
           }
         }
         
-        if (fieldTerms != null) {
-          final int v = fieldTerms.getDocCount();
-          if (v != -1 && visitedDocs.cardinality() != v) {
-            throw new RuntimeException("docCount for field " + field + "=" + v + " != recomputed docCount=" + visitedDocs.cardinality());
-          }
+        final int v = fieldTerms.getDocCount();
+        if (v != -1 && visitedDocs.cardinality() != v) {
+          throw new RuntimeException("docCount for field " + field + "=" + v + " != recomputed docCount=" + visitedDocs.cardinality());
         }
         
         // Test seek to last term:
@@ -1356,6 +1542,9 @@ public class CheckIndex implements Close
           if (termsEnum.seekCeil(lastTerm.get()) != TermsEnum.SeekStatus.FOUND) { 
             throw new RuntimeException("seek to last term " + lastTerm + " failed");
           }
+          if (termsEnum.term().equals(lastTerm.get()) == false) {
+            throw new RuntimeException("seek to last term " + lastTerm.get() + " returned FOUND but seeked to the wrong term " + termsEnum.term());
+          }
           
           int expectedDocFreq = termsEnum.docFreq();
           PostingsEnum d = termsEnum.postings(null, null, PostingsEnum.NONE);
@@ -1364,21 +1553,21 @@ public class CheckIndex implements Close
             docFreq++;
           }
           if (docFreq != expectedDocFreq) {
-            throw new RuntimeException("docFreq for last term " + lastTerm + "=" + expectedDocFreq + " != recomputed docFreq=" + docFreq);
+            throw new RuntimeException("docFreq for last term " + lastTerm.toBytesRef() + "=" + expectedDocFreq + " != recomputed docFreq=" + docFreq);
           }
         }
         
         // check unique term count
         long termCount = -1;
         
-        if ((status.delTermCount+status.termCount)-termCountStart > 0) {
+        if (fieldTermCount > 0) {
           termCount = fields.terms(field).size();
           
-          if (termCount != -1 && termCount != status.delTermCount + status.termCount - termCountStart) {
-            throw new RuntimeException("termCount mismatch " + (status.delTermCount + termCount) + " vs " + (status.termCount - termCountStart));
+          if (termCount != -1 && termCount != fieldTermCount) {
+            throw new RuntimeException("termCount mismatch " + termCount + " vs " + fieldTermCount);
           }
         }
-        
+
         // Test seeking by ord
         if (hasOrd && status.termCount-termCountStart > 0) {
           int seekCount = (int) Math.min(10000L, termCount);
@@ -1398,6 +1587,9 @@ public class CheckIndex implements Close
               if (termsEnum.seekCeil(seekTerms[i]) != TermsEnum.SeekStatus.FOUND) {
                 throw new RuntimeException("seek to existing term " + seekTerms[i] + " failed");
               }
+              if (termsEnum.term().equals(seekTerms[i]) == false) {
+                throw new RuntimeException("seek to existing term " + seekTerms[i] + " returned FOUND but seeked to the wrong term " + termsEnum.term());
+              }
               
               postings = termsEnum.postings(liveDocs, postings, PostingsEnum.NONE);
               if (postings == null) {

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java Thu Apr  2 15:37:39 2015
@@ -151,7 +151,6 @@ class FreqProxFields extends Fields {
     }
 
     public SeekStatus seekCeil(BytesRef text) {
-
       // TODO: we could instead keep the BytesRefHash
       // intact so this is a hash lookup
 
@@ -170,17 +169,19 @@ class FreqProxFields extends Fields {
         } else {
           // found:
           ord = mid;
+          assert term().compareTo(text) == 0;
           return SeekStatus.FOUND;
         }
       }
 
       // not found:
-      ord = lo + 1;
+      ord = lo;
       if (ord >= numTerms) {
         return SeekStatus.END;
       } else {
         int textStart = postingsArray.textStarts[sortedTermIDs[ord]];
         terms.bytePool.setBytesRef(scratch, textStart);
+        assert term().compareTo(text) > 0;
         return SeekStatus.NOT_FOUND;
       }
     }
@@ -309,7 +310,7 @@ class FreqProxFields extends Fields {
     final FreqProxPostingsArray postingsArray;
     final ByteSliceReader reader = new ByteSliceReader();
     final boolean readTermFreq;
-    int docID;
+    int docID = -1;
     int freq;
     boolean ended;
     int termID;
@@ -324,7 +325,7 @@ class FreqProxFields extends Fields {
       this.termID = termID;
       terms.initReader(reader, termID, 0);
       ended = false;
-      docID = 0;
+      docID = -1;
     }
 
     @Override
@@ -365,6 +366,9 @@ class FreqProxFields extends Fields {
 
     @Override
     public int nextDoc() throws IOException {
+      if (docID == -1) {
+        docID = 0;
+      }
       if (reader.eof()) {
         if (ended) {
           return NO_MORE_DOCS;
@@ -412,7 +416,7 @@ class FreqProxFields extends Fields {
     final ByteSliceReader reader = new ByteSliceReader();
     final ByteSliceReader posReader = new ByteSliceReader();
     final boolean readOffsets;
-    int docID;
+    int docID = -1;
     int freq;
     int pos;
     int startOffset;
@@ -436,7 +440,7 @@ class FreqProxFields extends Fields {
       terms.initReader(reader, termID, 0);
       terms.initReader(posReader, termID, 1);
       ended = false;
-      docID = 0;
+      docID = -1;
       posLeft = 0;
     }
 
@@ -452,6 +456,9 @@ class FreqProxFields extends Fields {
 
     @Override
     public int nextDoc() throws IOException {
+      if (docID == -1) {
+        docID = 0;
+      }
       while (posLeft != 0) {
         nextPosition();
       }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java Thu Apr  2 15:37:39 2015
@@ -49,6 +49,7 @@ final class MappingMultiPostingsEnum ext
     this.numSubs = postingsEnum.getNumSubs();
     this.subs = postingsEnum.getSubs();
     upto = -1;
+    doc = -1;
     current = null;
     this.multiDocsAndPositionsEnum = postingsEnum;
     return this;

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/TermContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/TermContext.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/TermContext.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/TermContext.java Thu Apr  2 15:37:39 2015
@@ -17,6 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
@@ -165,4 +166,30 @@ public final class TermContext {
   public void setDocFreq(int docFreq) {
     this.docFreq = docFreq;
   }
-}
\ No newline at end of file
+
+  /** Returns true if all terms stored here are real (e.g., not auto-prefix terms).
+   *
+   *  @lucene.internal */
+  public boolean hasOnlyRealTerms() {
+    for(TermState termState : states) {
+      if (termState instanceof BlockTermState && ((BlockTermState) termState).isRealTerm == false) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("TermContext\n");
+    for(TermState termState : states) {
+      sb.append("  state=");
+      sb.append(termState.toString());
+      sb.append('\n');
+    }
+
+    return sb.toString();
+  }
+}

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/index/Terms.java Thu Apr  2 15:37:39 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
+import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
@@ -42,17 +43,23 @@ public abstract class Terms {
    *  implementation can do so. */
   public abstract TermsEnum iterator(TermsEnum reuse) throws IOException;
 
-  /** Returns a TermsEnum that iterates over all terms that
-   *  are accepted by the provided {@link
+  /** Returns a TermsEnum that iterates over all terms and
+   *  documents that are accepted by the provided {@link
    *  CompiledAutomaton}.  If the <code>startTerm</code> is
-   *  provided then the returned enum will only accept terms
+   *  provided then the returned enum will only return terms
    *  {@code > startTerm}, but you still must call
    *  next() first to get to the first term.  Note that the
    *  provided <code>startTerm</code> must be accepted by
    *  the automaton.
    *
    * <p><b>NOTE</b>: the returned TermsEnum cannot
-   * seek</p>. */
+   * seek</p>.
+   *
+   *  <p><b>NOTE</b>: the terms dictionary is free to
+   *  return arbitrary terms as long as the resulted visited
+   *  docs is the same.  E.g., {@link BlockTreeTermsWriter}
+   *  creates auto-prefix terms during indexing to reduce the
+   *  number of terms visited. */
   public TermsEnum intersect(CompiledAutomaton compiled, final BytesRef startTerm) throws IOException {
     
     // TODO: could we factor out a common interface b/w
@@ -64,13 +71,17 @@ public abstract class Terms {
     // TODO: eventually we could support seekCeil/Exact on
     // the returned enum, instead of only being able to seek
     // at the start
+
+    TermsEnum termsEnum = iterator(null);
+
     if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
       throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
     }
+
     if (startTerm == null) {
-      return new AutomatonTermsEnum(iterator(null), compiled);
+      return new AutomatonTermsEnum(termsEnum, compiled);
     } else {
-      return new AutomatonTermsEnum(iterator(null), compiled) {
+      return new AutomatonTermsEnum(termsEnum, compiled) {
         @Override
         protected BytesRef nextSeekTerm(BytesRef term) throws IOException {
           if (term == null) {

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/AutomatonQuery.java Thu Apr  2 15:37:39 2015
@@ -99,6 +99,7 @@ public class AutomatonQuery extends Mult
     super(term.field());
     this.term = term;
     this.automaton = automaton;
+    // TODO: we could take isFinite too, to save a bit of CPU in CompiledAutomaton ctor?:
     this.compiled = new CompiledAutomaton(automaton, null, true, maxDeterminizedStates, isBinary);
   }
 

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/PrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/PrefixQuery.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/PrefixQuery.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/PrefixQuery.java Thu Apr  2 15:37:39 2015
@@ -17,12 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.automaton.Automaton;
@@ -33,6 +28,7 @@ import org.apache.lucene.util.automaton.
  * <p>This query uses the {@link
  * MultiTermQuery#CONSTANT_SCORE_REWRITE}
  * rewrite method. */
+
 public class PrefixQuery extends AutomatonQuery {
 
   /** Constructs a query for terms starting with <code>prefix</code>. */

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/ScoringRewrite.java Thu Apr  2 15:37:39 2015
@@ -18,19 +18,19 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
+
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.MultiTermQuery.RewriteMethod;
-
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 
 /** 
  * Base rewrite method that translates each term into a query, and keeps
@@ -112,7 +112,7 @@ public abstract class ScoringRewrite<Q e
       for (int i = 0; i < size; i++) {
         final int pos = sort[i];
         final Term term = new Term(query.getField(), col.terms.get(pos, new BytesRef()));
-        assert reader.docFreq(term) == termStates[pos].docFreq();
+        assert termStates[pos].hasOnlyRealTerms() == false || reader.docFreq(term) == termStates[pos].docFreq();
         addClause(result, term, termStates[pos].docFreq(), query.getBoost() * boost[pos], termStates[pos]);
       }
     }
@@ -137,7 +137,7 @@ public abstract class ScoringRewrite<Q e
       final int e = terms.add(bytes);
       final TermState state = termsEnum.termState();
       assert state != null; 
-      if (e < 0 ) {
+      if (e < 0) {
         // duplicate term: update docFreq
         final int pos = (-e)-1;
         array.termState[pos].register(state, readerContext.ord, termsEnum.docFreq(), termsEnum.totalTermFreq());

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/search/TermRangeQuery.java Thu Apr  2 15:37:39 2015
@@ -17,22 +17,17 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
+import org.apache.lucene.util.automaton.Automata;
+import org.apache.lucene.util.automaton.Automaton;
 
 /**
  * A Query that matches documents within an range of terms.
  *
  * <p>This query matches the documents looking for terms that fall into the
- * supplied range according to {@link
- * Byte#compareTo(Byte)}. It is not intended
- * for numerical ranges; use {@link NumericRangeQuery} instead.
+ * supplied range according to {@link BytesRef#compareTo(BytesRef)}.
  *
  * <p>This query uses the {@link
  * MultiTermQuery#CONSTANT_SCORE_REWRITE}
@@ -40,12 +35,11 @@ import org.apache.lucene.util.ToStringUt
  * @since 2.9
  */
 
-public class TermRangeQuery extends MultiTermQuery {
-  private BytesRef lowerTerm;
-  private BytesRef upperTerm;
-  private boolean includeLower;
-  private boolean includeUpper;
-
+public class TermRangeQuery extends AutomatonQuery {
+  private final BytesRef lowerTerm;
+  private final BytesRef upperTerm;
+  private final boolean includeLower;
+  private final boolean includeUpper;
 
   /**
    * Constructs a query selecting all terms greater/equal than <code>lowerTerm</code>
@@ -70,13 +64,28 @@ public class TermRangeQuery extends Mult
    *          included in the range.
    */
   public TermRangeQuery(String field, BytesRef lowerTerm, BytesRef upperTerm, boolean includeLower, boolean includeUpper) {
-    super(field);
+    super(new Term(field, lowerTerm), toAutomaton(lowerTerm, upperTerm, includeLower, includeUpper), Integer.MAX_VALUE, true);
     this.lowerTerm = lowerTerm;
     this.upperTerm = upperTerm;
     this.includeLower = includeLower;
     this.includeUpper = includeUpper;
   }
 
+  public static Automaton toAutomaton(BytesRef lowerTerm, BytesRef upperTerm, boolean includeLower, boolean includeUpper) {
+
+    if (lowerTerm == null) {
+      // makeBinaryInterval is more picky than we are:
+      includeLower = true;
+    }
+
+    if (upperTerm == null) {
+      // makeBinaryInterval is more picky than we are:
+      includeUpper = true;
+    }
+
+    return Automata.makeBinaryInterval(lowerTerm, includeLower, upperTerm, includeUpper);
+  }
+
   /**
    * Factory that creates a new TermRangeQuery using Strings for term text.
    */
@@ -98,37 +107,22 @@ public class TermRangeQuery extends Mult
   /** Returns <code>true</code> if the upper endpoint is inclusive */
   public boolean includesUpper() { return includeUpper; }
   
-  @Override
-  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    if (lowerTerm != null && upperTerm != null && lowerTerm.compareTo(upperTerm) > 0) {
-      return TermsEnum.EMPTY;
-    }
-    
-    TermsEnum tenum = terms.iterator(null);
-    
-    if ((lowerTerm == null || (includeLower && lowerTerm.length == 0)) && upperTerm == null) {
-      return tenum;
-    }
-    return new TermRangeTermsEnum(tenum,
-        lowerTerm, upperTerm, includeLower, includeUpper);
-  }
-
   /** Prints a user-readable version of this query. */
   @Override
   public String toString(String field) {
-      StringBuilder buffer = new StringBuilder();
-      if (!getField().equals(field)) {
-          buffer.append(getField());
-          buffer.append(":");
-      }
-      buffer.append(includeLower ? '[' : '{');
-      // TODO: all these toStrings for queries should just output the bytes, it might not be UTF-8!
-      buffer.append(lowerTerm != null ? ("*".equals(Term.toString(lowerTerm)) ? "\\*" : Term.toString(lowerTerm))  : "*");
-      buffer.append(" TO ");
-      buffer.append(upperTerm != null ? ("*".equals(Term.toString(upperTerm)) ? "\\*" : Term.toString(upperTerm)) : "*");
-      buffer.append(includeUpper ? ']' : '}');
-      buffer.append(ToStringUtils.boost(getBoost()));
-      return buffer.toString();
+    StringBuilder buffer = new StringBuilder();
+    if (!getField().equals(field)) {
+      buffer.append(getField());
+      buffer.append(":");
+    }
+    buffer.append(includeLower ? '[' : '{');
+    // TODO: all these toStrings for queries should just output the bytes, it might not be UTF-8!
+    buffer.append(lowerTerm != null ? ("*".equals(Term.toString(lowerTerm)) ? "\\*" : Term.toString(lowerTerm))  : "*");
+    buffer.append(" TO ");
+    buffer.append(upperTerm != null ? ("*".equals(Term.toString(upperTerm)) ? "\\*" : Term.toString(upperTerm)) : "*");
+    buffer.append(includeUpper ? ']' : '}');
+    buffer.append(ToStringUtils.boost(getBoost()));
+    return buffer.toString();
   }
 
   @Override
@@ -167,5 +161,4 @@ public class TermRangeQuery extends Mult
       return false;
     return true;
   }
-
 }

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automata.java Thu Apr  2 15:37:39 2015
@@ -72,6 +72,18 @@ final public class Automata {
     a.finishState();
     return a;
   }
+
+  /**
+   * Returns a new (deterministic) automaton that accepts all binary terms.
+   */
+  public static Automaton makeAnyBinary() {
+    Automaton a = new Automaton();
+    int s = a.createState();
+    a.setAccept(s, true);
+    a.addTransition(s, s, 0, 255);
+    a.finishState();
+    return a;
+  }
   
   /**
    * Returns a new (deterministic) automaton that accepts any single codepoint.
@@ -204,8 +216,172 @@ final public class Automata {
     return s;
   }
 
+  /** Creates a new deterministic, minimal automaton accepting
+   *  all binary terms in the specified interval.  Note that unlike
+   *  {@link #makeDecimalInterval}, the returned automaton is infinite,
+   *  because terms behave like floating point numbers leading with
+   *  a decimal point.  However, in the special case where min == max,
+   *  and both are inclusive, the automata will be finite and accept
+   *  exactly one term. */
+  public static Automaton makeBinaryInterval(BytesRef min, boolean minInclusive, BytesRef max, boolean maxInclusive) {
+
+    if (min == null && minInclusive == false) {
+      throw new IllegalArgumentException("minInclusive must be true when min is null (open ended)");
+    }
+
+    if (max == null && maxInclusive == false) {
+      throw new IllegalArgumentException("maxInclusive must be true when max is null (open ended)");
+    }
+
+    if (min != null && min.length == 0 && minInclusive == true) {
+      // Silly empty string corner case:
+      min = null;
+    }
+
+    if (min == null) {
+      if (max == null) {
+        // Accepts all terms:
+        return makeAnyBinary();
+      }
+      min = new BytesRef();
+      minInclusive = true;
+    }
+    int cmp;
+    if (max != null) {
+      cmp = min.compareTo(max);
+    } else {
+      cmp = -1;
+    }
+    if (cmp == 0) {
+      if (minInclusive == false || maxInclusive == false) {
+        return makeEmpty();
+      } else {
+        return makeBinary(min);
+      }
+    } else if (cmp > 0) {
+      // max > min
+      return makeEmpty();
+    }
+
+    Automaton a = new Automaton();
+    int startState = a.createState();
+    int sinkState = a.createState();
+    a.setAccept(sinkState, true);
+
+    // This state accepts all suffixes:
+    a.addTransition(sinkState, sinkState, 0, 255);
+
+    boolean equalPrefix = true;
+    int lastState = startState;
+    int firstMaxState = -1;
+    int sharedPrefixLength = 0;
+    for(int i=0;i<min.length;i++) {
+      int minLabel = min.bytes[min.offset+i] & 0xff;
+
+      int maxLabel;
+      if (max != null && equalPrefix && i < max.length) {
+        maxLabel = max.bytes[max.offset+i] & 0xff;
+      } else {
+        maxLabel = -1;
+      }
+
+      int nextState;
+      if (minInclusive && i == min.length-1 && (equalPrefix == false || minLabel != maxLabel)) {
+        nextState = sinkState;
+      } else {
+        nextState = a.createState();
+      }
+
+      if (equalPrefix) {
+
+        if (minLabel == maxLabel) {
+          // Still in shared prefix
+          a.addTransition(lastState, nextState, minLabel);
+        } else if (max == null) {
+          equalPrefix = false;
+          sharedPrefixLength = 0;
+          a.addTransition(lastState, sinkState, minLabel+1, 0xff);
+          a.addTransition(lastState, nextState, minLabel);
+        } else {
+          // This is the first point where min & max diverge:
+          assert maxLabel > minLabel;
+
+          a.addTransition(lastState, nextState, minLabel);
+
+          if (maxLabel > minLabel + 1) {
+            a.addTransition(lastState, sinkState, minLabel+1, maxLabel-1);
+          }
+
+          // Now fork off path for max:
+          if (maxInclusive || i < max.length-1) {
+            firstMaxState = a.createState();
+            if (i < max.length-1) {
+              a.setAccept(firstMaxState, true);
+            }
+            a.addTransition(lastState, firstMaxState, maxLabel);
+          }
+          equalPrefix = false;
+          sharedPrefixLength = i;
+        }
+      } else {
+        // OK, already diverged:
+        a.addTransition(lastState, nextState, minLabel);
+        if (minLabel < 255) {
+          a.addTransition(lastState, sinkState, minLabel+1, 255);
+        }
+      }
+      lastState = nextState;
+    }
+
+    // Accept any suffix appended to the min term:
+    if (equalPrefix == false && lastState != sinkState && lastState != startState) {
+      a.addTransition(lastState, sinkState, 0, 255);
+    }
+
+    if (minInclusive) {
+      // Accept exactly the min term:
+      a.setAccept(lastState, true);
+    }
+
+    if (max != null) {
+
+      // Now do max:
+      if (firstMaxState == -1) {
+        // Min was a full prefix of max
+        sharedPrefixLength = min.length;
+      } else {
+        lastState = firstMaxState;
+        sharedPrefixLength++;
+      }
+      for(int i=sharedPrefixLength;i<max.length;i++) {
+        int maxLabel = max.bytes[max.offset+i]&0xff;
+        if (maxLabel > 0) {
+          a.addTransition(lastState, sinkState, 0, maxLabel-1);
+        }
+        if (maxInclusive || i < max.length-1) {
+          int nextState = a.createState();
+          if (i < max.length-1) {
+            a.setAccept(nextState, true);
+          }
+          a.addTransition(lastState, nextState, maxLabel);
+          lastState = nextState;
+        }
+      }
+
+      if (maxInclusive) {
+        a.setAccept(lastState, true);
+      }
+    }
+
+    a.finishState();
+
+    assert a.isDeterministic(): a.toDot();
+
+    return a;
+  }
+
   /**
-   * Returns a new automaton that accepts strings representing decimal
+   * Returns a new automaton that accepts strings representing decimal (base 10)
    * non-negative integers in the given interval.
    * 
    * @param min minimal value of interval
@@ -218,7 +394,7 @@ final public class Automata {
    *              interval cannot be expressed with the given fixed number of
    *              digits
    */
-  public static Automaton makeInterval(int min, int max, int digits)
+  public static Automaton makeDecimalInterval(int min, int max, int digits)
       throws IllegalArgumentException {
     String x = Integer.toString(min);
     String y = Integer.toString(max);
@@ -275,7 +451,30 @@ final public class Automata {
     for (int i = 0, cp = 0; i < s.length(); i += Character.charCount(cp)) {
       int state = a.createState();
       cp = s.codePointAt(i);
-      a.addTransition(lastState, state, cp, cp);
+      a.addTransition(lastState, state, cp);
+      lastState = state;
+    }
+
+    a.setAccept(lastState, true);
+    a.finishState();
+
+    assert a.isDeterministic();
+    assert Operations.hasDeadStates(a) == false;
+
+    return a;
+  }
+
+  /**
+   * Returns a new (deterministic) automaton that accepts the single given
+   * binary term.
+   */
+  public static Automaton makeBinary(BytesRef term) {
+    Automaton a = new Automaton();
+    int lastState = a.createState();
+    for (int i=0;i<term.length;i++) {
+      int state = a.createState();
+      int label = term.bytes[term.offset+i] & 0xff;
+      a.addTransition(lastState, state, label);
       lastState = state;
     }
 

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java Thu Apr  2 15:37:39 2015
@@ -491,11 +491,50 @@ public class Automaton implements Accoun
   public void getNextTransition(Transition t) {
     // Make sure there is still a transition left:
     assert (t.transitionUpto+3 - states[2*t.source]) <= 3*states[2*t.source+1];
+
+    // Make sure transitions are in fact sorted:
+    assert transitionSorted(t);
+
     t.dest = transitions[t.transitionUpto++];
     t.min = transitions[t.transitionUpto++];
     t.max = transitions[t.transitionUpto++];
   }
 
+  private boolean transitionSorted(Transition t) {
+
+    int upto = t.transitionUpto;
+    if (upto == states[2*t.source]) {
+      // Transition isn't initialzed yet (this is the first transition); don't check:
+      return true;
+    }
+
+    int nextDest = transitions[upto];
+    int nextMin = transitions[upto+1];
+    int nextMax = transitions[upto+2];
+    if (nextMin > t.min) {
+      return true;
+    } else if (nextMin < t.min) {
+      return false;
+    }
+
+    // Min is equal, now test max:
+    if (nextMax > t.max) {
+      return true;
+    } else if (nextMax < t.max) {
+      return false;
+    }
+
+    // Max is also equal, now test dest:
+    if (nextDest > t.dest) {
+      return true;
+    } else if (nextDest < t.dest) {
+      return false;
+    }
+
+    // We should never see fully equal transitions here:
+    return false;
+  }
+
   /** Fill the provided {@link Transition} with the index'th
    *  transition leaving the specified state. */
   public void getTransition(int state, int index, Transition t) {
@@ -565,7 +604,7 @@ public class Automaton implements Accoun
       //System.out.println("toDot: state " + state + " has " + numTransitions + " transitions; t.nextTrans=" + t.transitionUpto);
       for(int i=0;i<numTransitions;i++) {
         getNextTransition(t);
-        //System.out.println("  t.nextTrans=" + t.transitionUpto);
+        //System.out.println("  t.nextTrans=" + t.transitionUpto + " t=" + t);
         assert t.max >= t.min;
         b.append("  ");
         b.append(state);

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/ByteRunAutomaton.java Thu Apr  2 15:37:39 2015
@@ -28,8 +28,8 @@ public class ByteRunAutomaton extends Ru
   }
   
   /** expert: if utf8 is true, the input is already byte-based */
-  public ByteRunAutomaton(Automaton a, boolean utf8, int maxDeterminizedStates) {
-    super(utf8 ? a : new UTF32ToUTF8().convert(a), 256, true, maxDeterminizedStates);
+  public ByteRunAutomaton(Automaton a, boolean isBinary, int maxDeterminizedStates) {
+    super(isBinary ? a : new UTF32ToUTF8().convert(a), 256, true, maxDeterminizedStates);
   }
 
   /**

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java Thu Apr  2 15:37:39 2015
@@ -90,12 +90,41 @@ public class CompiledAutomaton {
    */
   public final Boolean finite;
 
+  /** Which state, if any, accepts all suffixes, else -1. */
+  public final int sinkState;
+
   /** Create this, passing simplify=true and finite=null, so that we try
    *  to simplify the automaton and determine if it is finite. */
   public CompiledAutomaton(Automaton automaton) {
     this(automaton, null, true);
   }
 
+  /** Returns sink state, if present, else -1. */
+  private static int findSinkState(Automaton automaton) {
+    int numStates = automaton.getNumStates();
+    Transition t = new Transition();
+    int foundState = -1;
+    for (int s=0;s<numStates;s++) {
+      if (automaton.isAccept(s)) {
+        int count = automaton.initTransition(s, t);
+        boolean isSinkState = false;
+        for(int i=0;i<count;i++) {
+          automaton.getNextTransition(t);
+          if (t.dest == s && t.min == 0 && t.max == 0xff) {
+            isSinkState = true;
+            break;
+          }
+        }
+        if (isSinkState) {
+          foundState = s;
+          break;
+        }
+      }
+    }
+
+    return foundState;
+  }
+
   /** Create this.  If finite is null, we use {@link Operations#isFinite}
    *  to determine whether it is finite.  If simplify is true, we run
    *  possibly expensive operations to determine if the automaton is one
@@ -134,6 +163,7 @@ public class CompiledAutomaton {
         runAutomaton = null;
         this.automaton = null;
         this.finite = null;
+        sinkState = -1;
         return;
       }
 
@@ -154,6 +184,7 @@ public class CompiledAutomaton {
         runAutomaton = null;
         this.automaton = null;
         this.finite = null;
+        sinkState = -1;
         return;
       }
 
@@ -174,7 +205,7 @@ public class CompiledAutomaton {
         } else {
           term = new BytesRef(UnicodeUtil.newString(singleton.ints, singleton.offset, singleton.length));
         }
-
+        sinkState = -1;
         return;
       }
     }
@@ -202,7 +233,8 @@ public class CompiledAutomaton {
     if (this.finite) {
       commonSuffixRef = null;
     } else {
-      // NOTE: this is a very costly operation!  We should test if it's really warranted in practice...
+      // NOTE: this is a very costly operation!  We should test if it's really warranted in practice... we could do a fast match
+      // by looking for a sink state (which means it has no common suffix).  Or maybe we shouldn't do it when simplify is false?:
       BytesRef suffix = Operations.getCommonSuffixBytesRef(binary, maxDeterminizedStates);
       if (suffix.length == 0) {
         commonSuffixRef = null;
@@ -215,6 +247,10 @@ public class CompiledAutomaton {
     runAutomaton = new ByteRunAutomaton(binary, true, maxDeterminizedStates);
 
     this.automaton = runAutomaton.automaton;
+
+    // TODO: this is a bit fragile because if the automaton is not minimized there could be more than 1 sink state but auto-prefix will fail
+    // to run for those:
+    sinkState = findSinkState(this.automaton);
   }
 
   private Transition transition = new Transition();

Modified: lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java Thu Apr  2 15:37:39 2015
@@ -599,7 +599,7 @@ public class RegExp {
         a = aa;
         break;
       case REGEXP_INTERVAL:
-        a = Automata.makeInterval(min, max, digits);
+        a = Automata.makeDecimalInterval(min, max, digits);
         break;
     }
     return a;

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java Thu Apr  2 15:37:39 2015
@@ -117,8 +117,8 @@ public class TestAutomatonQuery extends
     assertAutomatonHits(2, Automata.makeString("doc"));
     assertAutomatonHits(1, Automata.makeChar('a'));
     assertAutomatonHits(2, Automata.makeCharRange('a', 'b'));
-    assertAutomatonHits(2, Automata.makeInterval(1233, 2346, 0));
-    assertAutomatonHits(1, Automata.makeInterval(0, 2000, 0));
+    assertAutomatonHits(2, Automata.makeDecimalInterval(1233, 2346, 0));
+    assertAutomatonHits(1, Automata.makeDecimalInterval(0, 2000, 0));
     assertAutomatonHits(2, Operations.union(Automata.makeChar('a'),
         Automata.makeChar('b')));
     assertAutomatonHits(0, Operations.intersection(Automata
@@ -194,7 +194,6 @@ public class TestAutomatonQuery extends
     Automaton pfx = Automata.makeString("do");
     Automaton prefixAutomaton = Operations.concatenate(pfx, Automata.makeAnyString());
     AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), prefixAutomaton);
-    Terms terms = MultiFields.getTerms(searcher.getIndexReader(), FN);
     assertEquals(3, automatonQueryNrHits(aq));
   }
   

Modified: lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java?rev=1670929&r1=1670928&r2=1670929&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java (original)
+++ lucene/dev/branches/lucene6271/lucene/core/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java Thu Apr  2 15:37:39 2015
@@ -17,16 +17,19 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
@@ -34,8 +37,6 @@ import org.apache.lucene.util.LuceneTest
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import java.io.IOException;
-
 public class TestMultiTermQueryRewrites extends LuceneTestCase {
 
   static Directory dir, sdir1, sdir2;
@@ -152,14 +153,27 @@ public class TestMultiTermQueryRewrites
     final MultiTermQuery mtq = new MultiTermQuery("data") {
       @Override
       protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-        return new TermRangeTermsEnum(terms.iterator(null), new BytesRef("2"), new BytesRef("7"), true, true) {
+        return new FilteredTermsEnum(terms.iterator(null)) {
+
           final BoostAttribute boostAtt =
             attributes().addAttribute(BoostAttribute.class);
         
           @Override
           protected AcceptStatus accept(BytesRef term) {
             boostAtt.setBoost(Float.parseFloat(term.utf8ToString()));
-            return super.accept(term);
+            if (term.length == 0) {
+              return AcceptStatus.NO;
+            }
+            char c = (char) (term.bytes[term.offset] & 0xff);
+            if (c >= '2') {
+              if (c <= '7') {
+                return AcceptStatus.YES;
+              } else {
+                return AcceptStatus.END;
+              }
+            } else {
+              return AcceptStatus.NO;
+            }
           }
         };
       }