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 2014/10/28 23:52:50 UTC

svn commit: r1635002 [2/4] - in /lucene/dev/branches/lucene6005/lucene: codecs/src/java/org/apache/lucene/codecs/blocktreeords/ codecs/src/java/org/apache/lucene/codecs/memory/ core/src/java/org/apache/lucene/codecs/ core/src/java/org/apache/lucene/cod...

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java Tue Oct 28 22:52:49 2014
@@ -23,6 +23,7 @@ import org.apache.lucene.index.DocsAndPo
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.ArrayUtil;
@@ -30,23 +31,38 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
-import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.RunAutomaton;
+import org.apache.lucene.util.automaton.Transition;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.Outputs;
 
-// NOTE: cannot seek!
+/** This is used to implement efficient {@link Terms#intersect} for
+ *  block-tree.  Note that it cannot seek, except for the initial term on
+ *  init.  It just "nexts" through the intersection of the automaton and
+ *  the terms.  It does not use the terms index at all: on init, it
+ *  loads the root block, and scans its way to the initial term.
+ *  Likewise, in next it scans until it finds a term that matches the
+ *  current automaton transition.  If the index has auto-prefix terms
+ *  (only for DOCS_ONLY fields currently) it will visit these terms
+ *  when possible and then skip the real terms that auto-prefix term
+ *  matched. */
+
 final class IntersectTermsEnum extends TermsEnum {
+
+  //static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
   final IndexInput in;
   final static Outputs<BytesRef> fstOutputs = ByteSequenceOutputs.getSingleton();
 
-  private IntersectTermsEnumFrame[] stack;
+  IntersectTermsEnumFrame[] stack;
       
   @SuppressWarnings({"rawtypes","unchecked"}) private FST.Arc<BytesRef>[] arcs = new FST.Arc[5];
 
   final RunAutomaton runAutomaton;
-  final CompiledAutomaton compiledAutomaton;
+  final Automaton automaton;
+  final BytesRef commonSuffix;
 
   private IntersectTermsEnumFrame currentFrame;
 
@@ -54,19 +70,33 @@ final class IntersectTermsEnum extends T
 
   private final FST.BytesReader fstReader;
 
+  private final boolean allowAutoPrefixTerms;
+
   final FieldReader fr;
 
+  /** Which state in the automaton accepts all possible suffixes. */
+  private final int sinkState;
+
   private BytesRef savedStartTerm;
       
+  /** True if we did return the current auto-prefix term */
+  private boolean useAutoPrefixTerm;
+
   // TODO: in some cases we can filter by length?  eg
   // regexp foo*bar must be at least length 6 bytes
-  public IntersectTermsEnum(FieldReader fr, CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
-    // if (DEBUG) {
-    //   System.out.println("\nintEnum.init seg=" + segment + " commonSuffix=" + brToString(compiled.commonSuffixRef));
-    // }
+  public IntersectTermsEnum(FieldReader fr, Automaton automaton, RunAutomaton runAutomaton, BytesRef commonSuffix, BytesRef startTerm, int sinkState) throws IOException {
+    //if (DEBUG) System.out.println("\nintEnum.init seg=" + fr.parent.segment + " commonSuffix=" + commonSuffix);
     this.fr = fr;
-    runAutomaton = compiled.runAutomaton;
-    compiledAutomaton = compiled;
+    this.sinkState = sinkState;
+
+    assert automaton != null;
+    assert runAutomaton != null;
+
+    //if (DEBUG) System.out.println("sinkState=" + sinkState + " AUTOMATON:\n" + automaton.toDot());
+    this.runAutomaton = runAutomaton;
+    this.allowAutoPrefixTerms = sinkState != -1;
+    this.automaton = automaton;
+    this.commonSuffix = commonSuffix;
     in = fr.parent.termsIn.clone();
     stack = new IntersectTermsEnumFrame[5];
     for(int idx=0;idx<stack.length;idx++) {
@@ -154,7 +184,7 @@ final class IntersectTermsEnum extends T
         
     f.fp = f.fpOrig = currentFrame.lastSubFP;
     f.prefix = currentFrame.prefix + currentFrame.suffix;
-    // if (DEBUG) System.out.println("    pushFrame state=" + state + " prefix=" + f.prefix);
+    //if (DEBUG) System.out.println("    pushFrame state=" + state + " prefix=" + f.prefix);
     f.setState(state);
 
     // Walk the arc through the index -- we only
@@ -233,7 +263,7 @@ final class IntersectTermsEnum extends T
   // arbitrary seekExact/Ceil.  Note that this is a
   // seekFloor!
   private void seekToStartTerm(BytesRef target) throws IOException {
-    //if (DEBUG) System.out.println("seek to startTerm=" + target.utf8ToString());
+    //if (DEBUG) System.out.println("seek to startTerm=" + target.utf8ToString() + " length=" + target.length);
     assert currentFrame.ord == 0;
     if (term.length < target.length) {
       term.bytes = ArrayUtil.grow(term.bytes, target.length);
@@ -242,23 +272,29 @@ final class IntersectTermsEnum extends T
     assert arc == currentFrame.arc;
 
     for(int idx=0;idx<=target.length;idx++) {
+      //if (DEBUG) System.out.println("cycle idx=" + idx);
 
       while (true) {
+        final int savNextEnt = currentFrame.nextEnt;
         final int savePos = currentFrame.suffixesReader.getPosition();
         final int saveStartBytePos = currentFrame.startBytePos;
         final int saveSuffix = currentFrame.suffix;
         final long saveLastSubFP = currentFrame.lastSubFP;
         final int saveTermBlockOrd = currentFrame.termState.termBlockOrd;
+        final boolean saveIsAutoPrefixTerm = currentFrame.isAutoPrefixTerm;
+
+        //if (DEBUG) System.out.println("    cycle isAutoPrefix=" + saveIsAutoPrefixTerm + " ent=" + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") prefix=" + currentFrame.prefix + " suffix=" + currentFrame.suffix + " firstLabel=" + (currentFrame.suffix == 0 ? "" : (currentFrame.suffixBytes[currentFrame.startBytePos])&0xff));
 
         final boolean isSubBlock = currentFrame.next();
 
-        //if (DEBUG) System.out.println("    cycle ent=" + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") prefix=" + currentFrame.prefix + " suffix=" + currentFrame.suffix + " isBlock=" + isSubBlock + " firstLabel=" + (currentFrame.suffix == 0 ? "" : (currentFrame.suffixBytes[currentFrame.startBytePos])&0xff));
         term.length = currentFrame.prefix + currentFrame.suffix;
         if (term.bytes.length < term.length) {
           term.bytes = ArrayUtil.grow(term.bytes, term.length);
         }
         System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
 
+        //if (DEBUG) System.out.println("      isSubBlock=" + isSubBlock + " term/prefix=" + brToString(term) + " saveIsAutoPrefixTerm=" + saveIsAutoPrefixTerm + " allowAutoPrefixTerms=" + allowAutoPrefixTerms);
+
         if (isSubBlock && StringHelper.startsWith(target, term)) {
           // Recurse
           //if (DEBUG) System.out.println("      recurse!");
@@ -266,9 +302,11 @@ final class IntersectTermsEnum extends T
           break;
         } else {
           final int cmp = term.compareTo(target);
+          //if (DEBUG) System.out.println("      cmp=" + cmp);
           if (cmp < 0) {
             if (currentFrame.nextEnt == currentFrame.entCount) {
               if (!currentFrame.isLastInFloor) {
+                // Advance to next floor block
                 //if (DEBUG) System.out.println("  load floorBlock");
                 currentFrame.loadNextFloorBlock();
                 continue;
@@ -279,19 +317,24 @@ final class IntersectTermsEnum extends T
             }
             continue;
           } else if (cmp == 0) {
+            if (allowAutoPrefixTerms == false && currentFrame.isAutoPrefixTerm) {
+              continue;
+            }
             //if (DEBUG) System.out.println("  return term=" + brToString(term));
             return;
-          } else {
+          } else if (allowAutoPrefixTerms || currentFrame.isAutoPrefixTerm == false) {
             // Fallback to prior entry: the semantics of
             // this method is that the first call to
             // next() will return the term after the
             // requested term
-            currentFrame.nextEnt--;
+            //if (DEBUG) System.out.println("    fallback prior entry");
+            currentFrame.nextEnt = savNextEnt;
             currentFrame.lastSubFP = saveLastSubFP;
             currentFrame.startBytePos = saveStartBytePos;
             currentFrame.suffix = saveSuffix;
             currentFrame.suffixesReader.setPosition(savePos);
             currentFrame.termState.termBlockOrd = saveTermBlockOrd;
+            currentFrame.isAutoPrefixTerm = saveIsAutoPrefixTerm;
             System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
             term.length = currentFrame.prefix + currentFrame.suffix;
             // If the last entry was a block we don't
@@ -310,77 +353,245 @@ final class IntersectTermsEnum extends T
   @Override
   public BytesRef next() throws IOException {
 
-    // if (DEBUG) {
-    //   System.out.println("\nintEnum.next seg=" + segment);
-    //   System.out.println("  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
-    // }
+    //if (DEBUG) {
+    //  System.out.println("\nintEnum.next seg=" + fr.parent.segment);
+    //  System.out.println("  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " outputPrefix=" + currentFrame.outputPrefix + " trans: " + currentFrame.transition + " useAutoPrefix=" + useAutoPrefixTerm);
+    //}
 
     nextTerm:
-    while(true) {
-      // Pop finished frames
-      while (currentFrame.nextEnt == currentFrame.entCount) {
-        if (!currentFrame.isLastInFloor) {
-          //if (DEBUG) System.out.println("    next-floor-block");
-          currentFrame.loadNextFloorBlock();
-          //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+    while (true) {
+
+      boolean isSubBlock;
+
+      if (useAutoPrefixTerm) {
+
+        assert currentFrame.isAutoPrefixTerm;
+        useAutoPrefixTerm = false;
+        currentFrame.termState.isRealTerm = true;
+
+        //if (DEBUG) System.out.println("    now scan beyond auto-prefix term=" + brToString(term) + " floorSuffixLeadEnd=" + Integer.toHexString(currentFrame.floorSuffixLeadEnd));
+        // If we last returned an auto-prefix term, we must now skip all
+        // actual terms sharing that prefix.  At most, that skipping
+        // requires popping one frame, but it can also require simply
+        // scanning ahead within the current frame.  This scanning will
+        // skip sub-blocks that contain many terms, which is why the
+        // optimization "works":
+        int floorSuffixLeadEnd = currentFrame.floorSuffixLeadEnd;
+        if (floorSuffixLeadEnd == -1) {
+          // An ordinary prefix, e.g. foo*
+          int prefix = currentFrame.prefix;
+          int suffix = currentFrame.suffix;
+          //if (DEBUG) System.out.println("    prefix=" + prefix + " suffix=" + suffix);
+          if (suffix == 0) {
+            //if (DEBUG) System.out.println("    pop frame & nextTerm");
+
+            // Easy case: the prefix term's suffix is the empty string,
+            // meaning the prefix corresponds to all terms in the
+            // current block, so we just pop this entire block:
+            if (currentFrame.ord == 0) {
+              //if (DEBUG) System.out.println("  return null");
+              return null;
+            }
+            currentFrame = stack[currentFrame.ord-1];
+            continue nextTerm;
+          } else {
+
+            // Just next() until we hit an entry that doesn't share this
+            // prefix.  The first next should be a sub-block sharing the
+            // same prefix, because if there are enough terms matching a
+            // given prefix to warrant an auto-prefix term, then there
+            // must also be enough to make a sub-block (assuming
+            // minItemsInPrefix > minItemsInBlock):
+            scanPrefix:
+            while (true) {
+              //if (DEBUG) System.out.println("    scan next");
+              if (currentFrame.nextEnt == currentFrame.entCount) {
+                if (currentFrame.isLastInFloor == false) {
+                  currentFrame.loadNextFloorBlock();
+                } else if (currentFrame.ord == 0) {
+                  //if (DEBUG) System.out.println("  return null0");
+                  return null;
+                } else {
+                  // Pop frame, which also means we've moved beyond this
+                  // auto-prefix term:
+                  //if (DEBUG) System.out.println("  pop; nextTerm");
+                  currentFrame = stack[currentFrame.ord-1];
+                  continue nextTerm;
+                }
+              }
+              isSubBlock = currentFrame.next();
+              //if (DEBUG) {
+              //  BytesRef suffixBytes = new BytesRef(currentFrame.suffix);
+              //  System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, suffixBytes.bytes, 0, currentFrame.suffix);
+              //  suffixBytes.length = currentFrame.suffix;
+              //  System.out.println("      currentFrame.suffix=" + brToString(suffixBytes));
+              //}
+              for(int i=0;i<suffix;i++) {
+                if (term.bytes[prefix+i] != currentFrame.suffixBytes[currentFrame.startBytePos+i]) {
+                  //if (DEBUG) System.out.println("      done; now stop scan");
+                  break scanPrefix;
+                }
+              }
+            }
+          }
         } else {
-          //if (DEBUG) System.out.println("  pop frame");
-          if (currentFrame.ord == 0) {
-            return null;
-          }
-          final long lastFP = currentFrame.fpOrig;
-          currentFrame = stack[currentFrame.ord-1];
-          assert currentFrame.lastSubFP == lastFP;
-          //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+          // Floor'd auto-prefix term; in this case we must skip all
+          // terms e.g. matching foo[a-m]*.  We are currently "on" fooa,
+          // which the automaton accepted (fooa* through foom*), and
+          // floorSuffixLeadEnd is m, so we must now scan to foon:
+          int prefix = currentFrame.prefix;
+          int suffix = currentFrame.suffix;
+
+          if (currentFrame.floorSuffixLeadStart == -1) {
+            suffix++;
+          }
+
+          //if (DEBUG) System.out.println("      prefix=" + prefix + " suffix=" + suffix);
+
+          if (suffix == 0) {
+
+            //if (DEBUG) System.out.println("  pop frame");
+
+            // This means current frame is fooa*, so we have to first
+            // pop the current frame, then scan in parent frame:
+            if (currentFrame.ord == 0) {
+              //if (DEBUG) System.out.println("  return null");
+              return null;
+            }
+            currentFrame = stack[currentFrame.ord-1];
+
+            // Current (parent) frame is now foo*, so now we just scan
+            // until the lead suffix byte is > floorSuffixLeadEnd
+            //assert currentFrame.prefix == prefix-1;
+            //prefix = currentFrame.prefix;
+
+            // In case when we pop, and the parent block is not just prefix-1, e.g. in block 417* on
+            // its first term = floor prefix term 41[7-9], popping to block 4*:
+            prefix = currentFrame.prefix;
+
+            suffix = term.length - currentFrame.prefix;
+          } else {
+            // No need to pop; just scan in currentFrame:
+          }
+
+          //if (DEBUG) System.out.println("    start scan: prefix=" + prefix + " suffix=" + suffix);
+
+          // Now we scan until the lead suffix byte is > floorSuffixLeadEnd
+          scanFloor:
+          while (true) {
+            //if (DEBUG) System.out.println("      scan next");
+            if (currentFrame.nextEnt == currentFrame.entCount) {
+              if (currentFrame.isLastInFloor == false) {
+                //if (DEBUG) System.out.println("      next floor block");
+                currentFrame.loadNextFloorBlock();
+              } else if (currentFrame.ord == 0) {
+                //if (DEBUG) System.out.println("  return null");
+                return null;
+              } else {
+                // Pop frame, which also means we've moved beyond this
+                // auto-prefix term:
+                currentFrame = stack[currentFrame.ord-1];
+                //if (DEBUG) System.out.println("      pop, now curFrame.prefix=" + currentFrame.prefix);
+                continue nextTerm;
+              }
+            }
+            isSubBlock = currentFrame.next();
+            //if (DEBUG) {
+            //  BytesRef suffixBytes = new BytesRef(currentFrame.suffix);
+            //  System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, suffixBytes.bytes, 0, currentFrame.suffix);
+            //  suffixBytes.length = currentFrame.suffix;
+            //  System.out.println("      currentFrame.suffix=" + brToString(suffixBytes));
+            //}
+            for(int i=0;i<suffix-1;i++) {
+              if (term.bytes[prefix+i] != currentFrame.suffixBytes[currentFrame.startBytePos+i]) {
+                //if (DEBUG) System.out.println("      done; now stop scan");
+                break scanFloor;
+              }
+            }
+            //if (DEBUG) {
+            //  if (currentFrame.suffix >= suffix) {
+            //    System.out.println("      cmp label=" + Integer.toHexString(currentFrame.suffixBytes[currentFrame.startBytePos+suffix-1]) + " vs " + floorSuffixLeadEnd);
+            //  }
+            //}
+            if (currentFrame.suffix >= suffix && (currentFrame.suffixBytes[currentFrame.startBytePos+suffix-1]&0xff) > floorSuffixLeadEnd) {
+              // Done scanning: we are now on the first term after all
+              // terms matched by this auto-prefix term
+              //if (DEBUG) System.out.println("      done; now stop scan");
+              break;
+            }
+          }
+        }
+      } else {
+        // Pop finished frames
+        while (currentFrame.nextEnt == currentFrame.entCount) {
+          if (!currentFrame.isLastInFloor) {
+            //if (DEBUG) System.out.println("    next-floor-block: trans: " + currentFrame.transition);
+            // Advance to next floor block
+            currentFrame.loadNextFloorBlock();
+            //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " outputPrefix=" + currentFrame.outputPrefix);
+            break;
+          } else {
+            //if (DEBUG) System.out.println("  pop frame");
+            if (currentFrame.ord == 0) {
+              //if (DEBUG) System.out.println("  return null");
+              return null;
+            }
+            final long lastFP = currentFrame.fpOrig;
+            currentFrame = stack[currentFrame.ord-1];
+            assert currentFrame.lastSubFP == lastFP;
+            //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " outputPrefix=" + currentFrame.outputPrefix);
+          }
         }
+
+        isSubBlock = currentFrame.next();
       }
 
-      final boolean isSubBlock = currentFrame.next();
-      // if (DEBUG) {
-      //   final BytesRef suffixRef = new BytesRef();
-      //   suffixRef.bytes = currentFrame.suffixBytes;
-      //   suffixRef.offset = currentFrame.startBytePos;
-      //   suffixRef.length = currentFrame.suffix;
-      //   System.out.println("    " + (isSubBlock ? "sub-block" : "term") + " " + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") suffix=" + brToString(suffixRef));
-      // }
+      //if (DEBUG) {
+      //  final BytesRef suffixRef = new BytesRef();
+      //  suffixRef.bytes = currentFrame.suffixBytes;
+      //  suffixRef.offset = currentFrame.startBytePos;
+      //  suffixRef.length = currentFrame.suffix;
+      //  System.out.println("    " + (isSubBlock ? "sub-block" : "term") + " " + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") suffix=" + brToString(suffixRef));
+      //}
 
       if (currentFrame.suffix != 0) {
+        // Advance where we are in the automaton to match what terms
+        // dict next'd to:
         final int label = currentFrame.suffixBytes[currentFrame.startBytePos] & 0xff;
+        //if (DEBUG) System.out.println("    move automaton to label=" + label + " vs curMax=" + currentFrame.curTransitionMax);
         while (label > currentFrame.curTransitionMax) {
           if (currentFrame.transitionIndex >= currentFrame.transitionCount-1) {
-            // Stop processing this frame -- no further
-            // matches are possible because we've moved
-            // beyond what the max transition will allow
-            //if (DEBUG) System.out.println("      break: trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]));
-
-            // sneaky!  forces a pop above
-            currentFrame.isLastInFloor = true;
-            currentFrame.nextEnt = currentFrame.entCount;
+            // Pop this frame: no further matches are possible because
+            // we've moved beyond what the max transition will allow
+            //if (DEBUG) System.out.println("      break: trans");
+            if (currentFrame.ord == 0) {
+              //if (DEBUG) System.out.println("  return null");
+              return null;
+            }
+            currentFrame = stack[currentFrame.ord-1];
             continue nextTerm;
           }
           currentFrame.transitionIndex++;
-          compiledAutomaton.automaton.getNextTransition(currentFrame.transition);
+          automaton.getNextTransition(currentFrame.transition);
           currentFrame.curTransitionMax = currentFrame.transition.max;
-          //if (DEBUG) System.out.println("      next trans=" + currentFrame.transitions[currentFrame.transitionIndex]);
+          //if (DEBUG) System.out.println("      next trans");
         }
       }
 
       // First test the common suffix, if set:
-      if (compiledAutomaton.commonSuffixRef != null && !isSubBlock) {
+      if (commonSuffix != null && !isSubBlock) {
         final int termLen = currentFrame.prefix + currentFrame.suffix;
-        if (termLen < compiledAutomaton.commonSuffixRef.length) {
+        if (termLen < commonSuffix.length) {
           // No match
-          // if (DEBUG) {
-          //   System.out.println("      skip: common suffix length");
-          // }
+          //if (DEBUG) System.out.println("      skip: common suffix length");
           continue nextTerm;
         }
 
         final byte[] suffixBytes = currentFrame.suffixBytes;
-        final byte[] commonSuffixBytes = compiledAutomaton.commonSuffixRef.bytes;
+        final byte[] commonSuffixBytes = commonSuffix.bytes;
 
-        final int lenInPrefix = compiledAutomaton.commonSuffixRef.length - currentFrame.suffix;
-        assert compiledAutomaton.commonSuffixRef.offset == 0;
+        final int lenInPrefix = commonSuffix.length - currentFrame.suffix;
+        assert commonSuffix.offset == 0;
         int suffixBytesPos;
         int commonSuffixBytesPos = 0;
 
@@ -394,24 +605,20 @@ final class IntersectTermsEnum extends T
           final int termBytesPosEnd = currentFrame.prefix;
           while (termBytesPos < termBytesPosEnd) {
             if (termBytes[termBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
-              // if (DEBUG) {
-              //   System.out.println("      skip: common suffix mismatch (in prefix)");
-              // }
+              //if (DEBUG) System.out.println("      skip: common suffix mismatch (in prefix)");
               continue nextTerm;
             }
           }
           suffixBytesPos = currentFrame.startBytePos;
         } else {
-          suffixBytesPos = currentFrame.startBytePos + currentFrame.suffix - compiledAutomaton.commonSuffixRef.length;
+          suffixBytesPos = currentFrame.startBytePos + currentFrame.suffix - commonSuffix.length;
         }
 
         // Test overlapping suffix part:
-        final int commonSuffixBytesPosEnd = compiledAutomaton.commonSuffixRef.length;
+        final int commonSuffixBytesPosEnd = commonSuffix.length;
         while (commonSuffixBytesPos < commonSuffixBytesPosEnd) {
           if (suffixBytes[suffixBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
-            // if (DEBUG) {
-            //   System.out.println("      skip: common suffix mismatch");
-            // }
+            //if (DEBUG) System.out.println("      skip: common suffix mismatch");
             continue nextTerm;
           }
         }
@@ -423,10 +630,17 @@ final class IntersectTermsEnum extends T
       // "temporarily" accepted, we just blindly .next()
       // until the limit
 
-      // See if the term prefix matches the automaton:
+      // TODO: for first iter of this loop can't we just use the current trans?  we already advanced it and confirmed it matches lead
+      // byte of the suffix
+
+      // See if the term suffix matches the automaton:
       int state = currentFrame.state;
+      int lastState = currentFrame.lastState;
+      //if (DEBUG) System.out.println("  a state=" + state + " curFrame.suffix.len=" + currentFrame.suffix + " curFrame.prefix=" + currentFrame.prefix);
       for (int idx=0;idx<currentFrame.suffix;idx++) {
-        state = runAutomaton.step(state,  currentFrame.suffixBytes[currentFrame.startBytePos+idx] & 0xff);
+        lastState = state;
+        //if (DEBUG) System.out.println("    step label=" + (char) (currentFrame.suffixBytes[currentFrame.startBytePos+idx] & 0xff));
+        state = runAutomaton.step(state, currentFrame.suffixBytes[currentFrame.startBytePos+idx] & 0xff);
         if (state == -1) {
           // No match
           //System.out.println("    no s=" + state);
@@ -436,16 +650,59 @@ final class IntersectTermsEnum extends T
         }
       }
 
+      //if (DEBUG) System.out.println("    after suffix: state=" + state + " lastState=" + lastState);
+
       if (isSubBlock) {
         // Match!  Recurse:
         //if (DEBUG) System.out.println("      sub-block match to state=" + state + "; recurse fp=" + currentFrame.lastSubFP);
         copyTerm();
         currentFrame = pushFrame(state);
-        //if (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+        currentFrame.lastState = lastState;
+        //xif (DEBUG) System.out.println("\n  frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
+      } else if (currentFrame.isAutoPrefixTerm) {
+        // We are on an auto-prefix term, meaning this term was compiled
+        // at indexing time, matching all terms sharing this prefix (or,
+        // a floor'd subset of them if that count was too high).  A
+        // prefix term represents a range of terms, so we now need to
+        // test whether, from the current state in the automaton, it
+        // accepts all terms in that range.  As long as it does, we can
+        // use this term and then later skip ahead past all terms in
+        // this range:
+        if (allowAutoPrefixTerms) {
+
+          if (currentFrame.floorSuffixLeadEnd == -1) {
+            // Simple prefix case
+            useAutoPrefixTerm = state == sinkState;
+          } else {
+            if (currentFrame.floorSuffixLeadStart == -1) {
+              // Must also accept the empty string in this case
+              if (automaton.isAccept(state)) {
+                //if (DEBUG) System.out.println("      state is accept");
+                useAutoPrefixTerm = acceptsSuffixRange(state, 0, currentFrame.floorSuffixLeadEnd);
+              }
+            } else {
+              useAutoPrefixTerm = acceptsSuffixRange(lastState, currentFrame.floorSuffixLeadStart, currentFrame.floorSuffixLeadEnd);
+            }
+          }
+
+          //if (DEBUG) System.out.println("  useAutoPrefixTerm=" + useAutoPrefixTerm);
+
+          if (useAutoPrefixTerm) {
+            copyTerm();
+            currentFrame.termState.isRealTerm = false;
+            //if (DEBUG) System.out.println("  return auto prefix term: " + brToString(term));
+            return term;
+          } else {
+            // We move onto the next term
+          }
+        } else {
+          // We are not allowed to use auto-prefix terms, so we just skip it
+        }
       } else if (runAutomaton.isAccept(state)) {
         copyTerm();
-        //if (DEBUG) System.out.println("      term match to state=" + state + "; return term=" + brToString(term));
+        //if (DEBUG) System.out.println("      term match to state=" + state);
         assert savedStartTerm == null || term.compareTo(savedStartTerm) > 0: "saveStartTerm=" + savedStartTerm.utf8ToString() + " term=" + term.utf8ToString();
+        //if (DEBUG) System.out.println("      return term=" + brToString(term));
         return term;
       } else {
         //System.out.println("    no s=" + state);
@@ -453,6 +710,41 @@ final class IntersectTermsEnum extends T
     }
   }
 
+  private final Transition transition = new Transition();
+
+  /** Returns true if, from this state, the automaton accepts any suffix
+   *  starting with a label between start and end, inclusive.  We just
+   *  look for a transition, matching this range, to the sink state.  */
+  private boolean acceptsSuffixRange(int state, int start, int end) {
+
+    //xif (DEBUG) System.out.println("    acceptsSuffixRange state=" + state + " start=" + start + " end=" + end);
+
+    int count = automaton.initTransition(state, transition);
+    //xif (DEBUG) System.out.println("      transCount=" + count);
+    //xif (DEBUG) System.out.println("      trans=" + transition);
+    for(int i=0;i<count;i++) {
+      automaton.getNextTransition(transition);
+      if (start >= transition.min && end <= transition.max && transition.dest == sinkState) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  // 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();
+    }
+  }
+
   private void copyTerm() {
     //System.out.println("      copyTerm cur.prefix=" + currentFrame.prefix + " cur.suffix=" + currentFrame.suffix + " first=" + (char) currentFrame.suffixBytes[currentFrame.startBytePos]);
     final int len = currentFrame.prefix + currentFrame.suffix;

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java Tue Oct 28 22:52:49 2014
@@ -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,102 @@ 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;
+      }
     }
   }
 
@@ -277,10 +360,10 @@ final class IntersectTermsEnumFrame {
 
       // stats
       termState.docFreq = statsReader.readVInt();
-      //if (DEBUG) System.out.println("    dF=" + state.docFreq);
+      //xif (DEBUG) System.out.println("    dF=" + state.docFreq);
       if (ite.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
         termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
-        //if (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
+        //xif (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
       }
       // metadata 
       for (int i = 0; i < ite.fr.longsSize; i++) {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java Tue Oct 28 22:52:49 2014
@@ -36,7 +36,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:
@@ -50,7 +52,7 @@ final class SegmentTermsEnum extends Ter
 
   private int targetBeforeCurrentLength;
 
-  // static boolean DEBUG = false;
+  //static boolean DEBUG = BlockTreeTermsWriter.DEBUG;
 
   private final ByteArrayDataInput scratchReader = new ByteArrayDataInput();
 
@@ -121,6 +123,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();
@@ -154,8 +158,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;
@@ -177,8 +183,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 {
@@ -296,6 +300,7 @@ final class SegmentTermsEnum extends Ter
     return true;
   }
 
+  /*
   // for debugging
   @SuppressWarnings("unused")
   static String brToString(BytesRef b) {
@@ -309,8 +314,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");
@@ -567,7 +579,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");
     }
@@ -577,7 +590,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);
     // }
 
@@ -619,7 +632,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;
@@ -649,7 +662,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;
@@ -735,7 +748,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;
@@ -745,6 +758,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);
@@ -752,7 +766,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 {
@@ -763,7 +777,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;
         }
@@ -778,7 +792,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++;
 
@@ -804,7 +818,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 {
@@ -908,7 +922,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) {
@@ -948,11 +964,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/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java Tue Oct 28 22:52:49 2014
@@ -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,73 @@ 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();
+        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 +495,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 +540,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 +617,6 @@ final class SegmentTermsEnumFrame {
           // keep scanning
 
           if (nextEnt == entCount) {
-            if (exactOnly) {
-              fillTerm();
-            }
             // We are done scanning this block
             break nextTerm;
           } else {
@@ -590,7 +669,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 +684,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 +746,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 +761,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/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/blocktree/Stats.java Tue Oct 28 22:52:49 2014
@@ -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/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java Tue Oct 28 22:52:49 2014
@@ -391,8 +391,10 @@ public final class Lucene50PostingsForma
   final static int VERSION_START = 0;
   final static int VERSION_CURRENT = VERSION_START;
 
-  private final int minTermBlockSize;
-  private final int maxTermBlockSize;
+  private final int minTemsInBlock;
+  private final int maxItemsInBlock;
+  private final int minItemsInAutoPrefix;
+  private final int maxItemsInAutoPrefix;
 
   /**
    * Fixed packed block size, number of integers encoded in 
@@ -404,19 +406,33 @@ public final class Lucene50PostingsForma
   /** Creates {@code Lucene50PostingsFormat} with default
    *  settings. */
   public Lucene50PostingsFormat() {
-    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE, 0, 0);
   }
 
   /** Creates {@code Lucene50PostingsFormat} with custom
    *  values for {@code minBlockSize} and {@code
-   *  maxBlockSize} passed to block terms dictionary.
+   *  maxBlockSize} and default values for {@code minItemsInAutoPrefix} and
+   *  {@code maxItemsInAutoPrefix}, passed to block tree terms dictionary.
    *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
-  public Lucene50PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
+  public Lucene50PostingsFormat(int minTemsInBlock, int maxItemsInBlock) {
+    this(minTemsInBlock, maxItemsInBlock, 0, 0);
+  }
+
+  /** Creates {@code Lucene50PostingsFormat} with custom
+   *  values for {@code minBlockSize}, {@code
+   *  maxBlockSize}, {@code minItemsInAutoPrefix} and {@code maxItemsInAutoPrefix}, passed
+   *  to block tree terms dictionary.
+   *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int,int,int) */
+  public Lucene50PostingsFormat(int minTemsInBlock, int maxItemsInBlock, int minItemsInAutoPrefix, int maxItemsInAutoPrefix) {
     super("Lucene50");
-    this.minTermBlockSize = minTermBlockSize;
-    assert minTermBlockSize > 1;
-    this.maxTermBlockSize = maxTermBlockSize;
-    assert minTermBlockSize <= maxTermBlockSize;
+    BlockTreeTermsWriter.validateSettings(minTemsInBlock,
+                                          maxItemsInBlock);
+    BlockTreeTermsWriter.validateAutoPrefixSettings(minItemsInAutoPrefix,
+                                                    maxItemsInAutoPrefix);
+    this.minTemsInBlock = minTemsInBlock;
+    this.maxItemsInBlock = maxItemsInBlock;
+    this.minItemsInAutoPrefix = minItemsInAutoPrefix;
+    this.maxItemsInAutoPrefix = maxItemsInAutoPrefix;
   }
 
   @Override
@@ -432,8 +448,10 @@ public final class Lucene50PostingsForma
     try {
       FieldsConsumer ret = new BlockTreeTermsWriter(state, 
                                                     postingsWriter,
-                                                    minTermBlockSize, 
-                                                    maxTermBlockSize);
+                                                    minTemsInBlock, 
+                                                    maxItemsInBlock,
+                                                    minItemsInAutoPrefix,
+                                                    maxItemsInAutoPrefix);
       success = true;
       return ret;
     } finally {

Modified: lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java?rev=1635002&r1=1635001&r2=1635002&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/java/org/apache/lucene/document/Document2.java Tue Oct 28 22:52:49 2014
@@ -24,7 +24,6 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.NumericTokenStream;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.FieldTypes.FieldType;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
@@ -81,6 +80,20 @@ public class Document2 implements IndexD
       return boost;
     }
 
+    private TokenStream getReusedBinaryTokenStream(BytesRef value, TokenStream reuse) {
+      BinaryTokenStream bts;
+      if (reuse != null) {
+        if (reuse instanceof BinaryTokenStream == false) {
+          FieldTypes.illegalState(fieldName, "should have had BinaryTokenStream for reuse, but got " + reuse);
+        }
+        bts = (BinaryTokenStream) reuse;
+      } else {
+        bts = new BinaryTokenStream();
+      }
+      bts.setValue(value);
+      return bts;
+    }
+
     @Override
     public TokenStream tokenStream(Analyzer analyzerIn, TokenStream reuse) throws IOException {
       Analyzer analyzer = fieldTypes.getIndexAnalyzer();
@@ -92,41 +105,13 @@ public class Document2 implements IndexD
       FieldTypes.FieldType fieldType = fieldTypes.getFieldType(fieldName);
       switch (fieldType.valueType) {
       case INT:
+        return getReusedBinaryTokenStream(intToBytes(((Number) value).intValue()), reuse);
       case FLOAT:
+        return getReusedBinaryTokenStream(intToBytes(Float.floatToIntBits(((Number) value).floatValue())), reuse);
       case LONG:
+        return getReusedBinaryTokenStream(longToBytes(((Number) value).longValue()), reuse);
       case DOUBLE:
-        NumericTokenStream nts;
-        if (reuse != null) {
-          if (reuse instanceof NumericTokenStream == false) {
-            FieldTypes.illegalState(fieldName, "should have had NumericTokenStream for reuse, but got " + reuse);
-          }
-          nts = (NumericTokenStream) reuse;
-          if (fieldType.numericPrecisionStep == null || nts.getPrecisionStep() != fieldType.numericPrecisionStep.intValue()) {
-            FieldTypes.illegalState(fieldName, "reused NumericTokenStream has precisionStep " + nts.getPrecisionStep() + ", which is different from FieldType's " + fieldType.numericPrecisionStep);
-          }
-        } else {
-          nts = new NumericTokenStream(fieldType.numericPrecisionStep);
-        }
-        // initialize value in TokenStream
-        final Number number = (Number) value;
-        switch (fieldType.valueType) {
-        case INT:
-          nts.setIntValue(number.intValue());
-          break;
-        case LONG:
-          nts.setLongValue(number.longValue());
-          break;
-        case FLOAT:
-          nts.setFloatValue(number.floatValue());
-          break;
-        case DOUBLE:
-          nts.setDoubleValue(number.doubleValue());
-          break;
-        default:
-          throw new AssertionError("Should never get here");
-        }
-        return nts;
-
+        return getReusedBinaryTokenStream(longToBytes(Double.doubleToLongBits(((Number) value).doubleValue())), reuse);
       case ATOM:
         if (value instanceof String) {
           StringTokenStream sts;
@@ -142,17 +127,7 @@ public class Document2 implements IndexD
           return sts;
         } else {
           assert value instanceof BytesRef;
-          BinaryTokenStream bts;
-          if (reuse != null) {
-            if (reuse instanceof BinaryTokenStream == false) {
-              FieldTypes.illegalState(fieldName, "should have had BinaryTokenStream for reuse, but got " + reuse);
-            }
-            bts = (BinaryTokenStream) reuse;
-          } else {
-            bts = new BinaryTokenStream();
-          }
-          bts.setValue((BytesRef) value);
-          return bts;
+          return getReusedBinaryTokenStream((BytesRef) value, reuse);
         }
 
       case BINARY:
@@ -453,4 +428,30 @@ public class Document2 implements IndexD
       }
     }
   }
+
+  static BytesRef intToBytes(int v) {
+    int sortableBits = v ^ 0x80000000;
+    BytesRef token = new BytesRef(4);
+    token.length = 4;
+    int index = 3;
+    while (index >= 0) {
+      token.bytes[index] = (byte) (sortableBits & 0xff);
+      index--;
+      sortableBits >>>= 8;
+    }
+    return token;
+  }
+
+  static BytesRef longToBytes(long v) {
+    long sortableBits = v ^ 0x8000000000000000L;
+    BytesRef token = new BytesRef(8);
+    token.length = 8;
+    int index = 7;
+    while (index >= 0) {
+      token.bytes[index] = (byte) (sortableBits & 0xff);
+      index--;
+      sortableBits >>>= 8;
+    }
+    return token;
+  }
 }