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

svn commit: r1670918 [2/5] - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/autoprefix/ codecs/src/resources/META-INF/services/ codecs/src/test/org/apache/lucene/codecs/autoprefix/ core/src/java/org/apache/lucene/codecs/ core/...

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java Thu Apr  2 15:05:48 2015
@@ -25,11 +25,13 @@ import org.apache.lucene.codecs.BlockTer
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.blocktree.AutoPrefixTermsWriter.PrefixTerm;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -87,6 +89,16 @@ import org.apache.lucene.util.packed.Pac
  * stride) each term's metadata for each set of terms
  * between two index terms.
  * <p>
+ *
+ * If {@code minItemsInAutoPrefix} is not zero, then for
+ * {@link IndexOptions#DOCS} fields we detect prefixes that match
+ * "enough" terms and insert auto-prefix terms into the index, which are
+ * used by {@link Terms#intersect}  at search time to speed up prefix
+ * and range queries.  Besides {@link Terms#intersect}, these
+ * auto-prefix terms are invisible to all other APIs (don't change terms
+ * stats, don't show up in normal {@link TermsEnum}s, etc.).
+ * <p>
+ *
  * Files:
  * <ul>
  *   <li><tt>.tim</tt>: <a href="#Termdictionary">Term Dictionary</a></li>
@@ -200,7 +212,9 @@ public final class BlockTreeTermsWriter
    *  #BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */
   public final static int DEFAULT_MAX_BLOCK_SIZE = 48;
 
-  // public final static boolean DEBUG = false;
+  //public static boolean DEBUG = false;
+  //public static boolean DEBUG2 = false;
+
   //private final static boolean SAVE_DOT_FILES = false;
 
   private final IndexOutput termsOut;
@@ -208,6 +222,8 @@ public final class BlockTreeTermsWriter
   final int maxDoc;
   final int minItemsInBlock;
   final int maxItemsInBlock;
+  final int minItemsInAutoPrefix;
+  final int maxItemsInAutoPrefix;
 
   final PostingsWriterBase postingsWriter;
   final FieldInfos fieldInfos;
@@ -244,23 +260,67 @@ public final class BlockTreeTermsWriter
   private final List<FieldMetaData> fields = new ArrayList<>();
 
   // private final String segment;
+  final FixedBitSet prefixDocs;
+
+  /** Reused in getAutoPrefixTermsEnum: */
+  final BitSetTermsEnum prefixFixedBitsTermsEnum;
+
+  /** Reused in getAutoPrefixTermsEnum: */
+  private TermsEnum prefixTermsEnum;
+
+  /** Reused in getAutoPrefixTermsEnum: */
+  private PostingsEnum prefixDocsEnum;
+
+  /** Create a new writer, using default values for auto-prefix terms. */
+  public BlockTreeTermsWriter(SegmentWriteState state,
+                              PostingsWriterBase postingsWriter,
+                              int minItemsInBlock,
+                              int maxItemsInBlock) throws IOException {
+    this(state, postingsWriter, minItemsInBlock, maxItemsInBlock, 0, 0);
+  }
 
   /** Create a new writer.  The number of items (terms or
    *  sub-blocks) per block will aim to be between
    *  minItemsPerBlock and maxItemsPerBlock, though in some
-   *  cases the blocks may be smaller than the min. */
+   *  cases the blocks may be smaller than the min.
+   *  For DOCS_ONLY fields, this terms dictionary will
+   *  insert automatically generated prefix terms for common
+   *  prefixes, as long as each prefix matches at least
+   *  {@code minItemsInAutoPrefix} other terms or prefixes,
+   *  and at most {@code maxItemsInAutoPrefix} other terms
+   *  or prefixes.  Set {@code minItemsInAutoPrefix} to 0
+   *  to disable auto-prefix terms. */
   public BlockTreeTermsWriter(SegmentWriteState state,
                               PostingsWriterBase postingsWriter,
                               int minItemsInBlock,
-                              int maxItemsInBlock)
+                              int maxItemsInBlock,
+                              int minItemsInAutoPrefix,
+                              int maxItemsInAutoPrefix)
     throws IOException
   {
-    validateSettings(minItemsInBlock, maxItemsInBlock);
+    validateSettings(minItemsInBlock,
+                     maxItemsInBlock);
 
-    this.maxDoc = state.segmentInfo.maxDoc();
-    this.fieldInfos = state.fieldInfos;
     this.minItemsInBlock = minItemsInBlock;
     this.maxItemsInBlock = maxItemsInBlock;
+
+    validateAutoPrefixSettings(minItemsInAutoPrefix,
+                               maxItemsInAutoPrefix);
+
+    if (minItemsInAutoPrefix != 0) {
+      // TODO: can we used compressed bitset instead?  that auto-upgrades if it's dense enough...
+      prefixDocs = new FixedBitSet(state.segmentInfo.maxDoc());
+      prefixFixedBitsTermsEnum = new BitSetTermsEnum(prefixDocs);
+    } else {
+      prefixDocs = null;
+      prefixFixedBitsTermsEnum = null;
+    }
+
+    this.minItemsInAutoPrefix = minItemsInAutoPrefix;
+    this.maxItemsInAutoPrefix = maxItemsInAutoPrefix;
+
+    this.maxDoc = state.segmentInfo.maxDoc();
+    this.fieldInfos = state.fieldInfos;
     this.postingsWriter = postingsWriter;
 
     final String termsName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_EXTENSION);
@@ -269,12 +329,13 @@ public final class BlockTreeTermsWriter
     IndexOutput indexOut = null;
     try {
       CodecUtil.writeIndexHeader(termsOut, BlockTreeTermsReader.TERMS_CODEC_NAME, BlockTreeTermsReader.VERSION_CURRENT,
-                                  state.segmentInfo.getId(), state.segmentSuffix);
+                                 state.segmentInfo.getId(), state.segmentSuffix);
 
       final String indexName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_INDEX_EXTENSION);
       indexOut = state.directory.createOutput(indexName, state.context);
       CodecUtil.writeIndexHeader(indexOut, BlockTreeTermsReader.TERMS_INDEX_CODEC_NAME, BlockTreeTermsReader.VERSION_CURRENT,
-                                   state.segmentInfo.getId(), state.segmentSuffix);
+                                 state.segmentInfo.getId(), state.segmentSuffix);
+      //segment = state.segmentInfo.name;
 
       postingsWriter.init(termsOut, state);                          // have consumer write its format/header
       
@@ -311,32 +372,107 @@ public final class BlockTreeTermsWriter
     }
   }
 
+  /** Throws {@code IllegalArgumentException} if any of these settings
+   *  is invalid. */
+  public static void validateAutoPrefixSettings(int minItemsInAutoPrefix,
+                                                int maxItemsInAutoPrefix) {
+    if (minItemsInAutoPrefix != 0) {
+      if (minItemsInAutoPrefix < 2) {
+        throw new IllegalArgumentException("minItemsInAutoPrefix must be at least 2; got minItemsInAutoPrefix=" + minItemsInAutoPrefix);
+      }
+      if (minItemsInAutoPrefix > maxItemsInAutoPrefix) {
+        throw new IllegalArgumentException("maxItemsInAutoPrefix must be >= minItemsInAutoPrefix; got maxItemsInAutoPrefix=" + maxItemsInAutoPrefix + " minItemsInAutoPrefix=" + minItemsInAutoPrefix);
+      }
+      if (2*(minItemsInAutoPrefix-1) > maxItemsInAutoPrefix) {
+        throw new IllegalArgumentException("maxItemsInAutoPrefix must be at least 2*(minItemsInAutoPrefix-1); got maxItemsInAutoPrefix=" + maxItemsInAutoPrefix + " minItemsInAutoPrefix=" + minItemsInAutoPrefix);
+      }
+    } else if (maxItemsInAutoPrefix != 0) {
+      throw new IllegalArgumentException("maxItemsInAutoPrefix must be 0 (disabled) when minItemsInAutoPrefix is 0");
+    }
+  }
+
   @Override
   public void write(Fields fields) throws IOException {
+    //if (DEBUG) System.out.println("\nBTTW.write seg=" + segment);
 
     String lastField = null;
     for(String field : fields) {
       assert lastField == null || lastField.compareTo(field) < 0;
       lastField = field;
 
+      //if (DEBUG) System.out.println("\nBTTW.write seg=" + segment + " field=" + field);
       Terms terms = fields.terms(field);
       if (terms == null) {
         continue;
       }
+      FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
 
-      TermsEnum termsEnum = terms.iterator(null);
+      // First pass to find all prefix terms we should compile into the index:
+      List<PrefixTerm> prefixTerms;
+      if (minItemsInAutoPrefix != 0) {
+        if (fieldInfo.getIndexOptions() != IndexOptions.DOCS) {
+          throw new IllegalStateException("ranges can only be indexed with IndexOptions.DOCS (field: " + fieldInfo.name + ")");
+        }
+        prefixTerms = new AutoPrefixTermsWriter(terms, minItemsInAutoPrefix, maxItemsInAutoPrefix).prefixes;
+        //if (DEBUG) {
+        //  for(PrefixTerm term : prefixTerms) {
+        //    System.out.println("field=" + fieldInfo.name + " PREFIX TERM: " + term);
+        //  }
+        //}
+      } else {
+        prefixTerms = null;
+      }
 
+      TermsEnum termsEnum = terms.iterator(null);
       TermsWriter termsWriter = new TermsWriter(fieldInfos.fieldInfo(field));
+      int prefixTermUpto = 0;
       while (true) {
         BytesRef term = termsEnum.next();
+        //if (DEBUG) System.out.println("BTTW: next term " + term);
+
+        // Insert (merge sort) next prefix term(s):
+        if (prefixTerms != null) {
+          while (prefixTermUpto < prefixTerms.size() && (term == null || prefixTerms.get(prefixTermUpto).compareTo(term) <= 0)) {
+            PrefixTerm prefixTerm = prefixTerms.get(prefixTermUpto);
+            //if (DEBUG) System.out.println("seg=" + segment + " field=" + fieldInfo.name + " NOW INSERT prefix=" + prefixTerm);
+            termsWriter.write(prefixTerm.term, getAutoPrefixTermsEnum(terms, prefixTerm), prefixTerm);
+            prefixTermUpto++;
+          }
+        }
+
         if (term == null) {
           break;
         }
-        termsWriter.write(term, termsEnum);
+
+        //if (DEBUG) System.out.println("write field=" + fieldInfo.name + " term=" + brToString(term));
+        termsWriter.write(term, termsEnum, null);
       }
 
+      assert prefixTerms == null || prefixTermUpto == prefixTerms.size();
+
       termsWriter.finish();
+
+      //if (DEBUG) System.out.println("\nBTTW.write done seg=" + segment + " field=" + field);
+    }
+  }
+
+  private TermsEnum getAutoPrefixTermsEnum(Terms terms, final PrefixTerm prefix) throws IOException {
+    assert prefixDocs != null;
+    prefixDocs.clear(0, prefixDocs.length());
+
+    prefixTermsEnum = prefix.getTermsEnum(terms.iterator(prefixTermsEnum));
+
+    //System.out.println("BTTW.getAutoPrefixTE: prefix=" + prefix);
+    while (prefixTermsEnum.next() != null) {
+      //System.out.println("    got term=" + prefixTermsEnum.term().utf8ToString());
+      //termCount++;
+      prefixDocsEnum = prefixTermsEnum.postings(null, prefixDocsEnum, 0);
+      //System.out.println("      " + prefixDocsEnum + " doc=" + prefixDocsEnum.docID());
+      prefixDocs.or(prefixDocsEnum);
     }
+
+    //System.out.println("  done terms: " + prefixDocs.cardinality() + " doc seen; " + termCount + " terms seen");
+    return prefixFixedBitsTermsEnum;
   }
   
   static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) {
@@ -356,30 +492,38 @@ public final class BlockTreeTermsWriter
     public final byte[] termBytes;
     // stats + metadata
     public final BlockTermState state;
+    // Non-null if this is an auto-prefix-term:
+    public final PrefixTerm prefixTerm;
+    public PendingTerm other;
 
-    public PendingTerm(BytesRef term, BlockTermState state) {
+    public PendingTerm(BytesRef term, BlockTermState state, PrefixTerm prefixTerm) {
       super(true);
       this.termBytes = new byte[term.length];
       System.arraycopy(term.bytes, term.offset, termBytes, 0, term.length);
       this.state = state;
+      this.prefixTerm = prefixTerm;
     }
 
     @Override
     public String toString() {
-      return brToString(termBytes);
+      return "TERM: " + brToString(termBytes);
     }
   }
 
   // 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();
+    if (b == null) {
+      return "(null)";
+    } else {
+      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();
+      }
     }
   }
 
@@ -410,7 +554,7 @@ public final class BlockTreeTermsWriter
 
     @Override
     public String toString() {
-      return "BLOCK: " + brToString(prefix);
+      return "BLOCK: prefix=" + brToString(prefix);
     }
 
     public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
@@ -493,6 +637,8 @@ public final class BlockTreeTermsWriter
   private final RAMOutputStream scratchBytes = new RAMOutputStream();
   private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
 
+  static final BytesRef EMPTY_BYTES_REF = new BytesRef();
+
   class TermsWriter {
     private final FieldInfo fieldInfo;
     private final int longsSize;
@@ -529,14 +675,11 @@ public final class BlockTreeTermsWriter
 
       assert count > 0;
 
-      /*
-      if (DEBUG) {
-        BytesRef br = new BytesRef(lastTerm.bytes);
-        br.offset = lastTerm.offset;
-        br.length = prefixLength;
-        System.out.println("writeBlocks: " + br.utf8ToString() + " count=" + count);
-      }
-      */
+      //if (DEBUG2) {
+      //  BytesRef br = new BytesRef(lastTerm.bytes());
+      //  br.length = prefixLength;
+      //  System.out.println("writeBlocks: seg=" + segment + " prefix=" + brToString(br) + " count=" + count);
+      //}
 
       // Root block better write all remaining pending entries:
       assert prefixLength > 0 || count == pending.size();
@@ -547,6 +690,7 @@ public final class BlockTreeTermsWriter
       // only points to sub-blocks in the terms index so we can avoid seeking
       // to it when we are looking for a term):
       boolean hasTerms = false;
+      boolean hasPrefixTerms = false;
       boolean hasSubBlocks = false;
 
       int start = pending.size()-count;
@@ -566,7 +710,7 @@ public final class BlockTreeTermsWriter
             // Suffix is 0, i.e. prefix 'foo' and term is
             // 'foo' so the term has empty string suffix
             // in this block
-            assert lastSuffixLeadLabel == -1;
+            assert lastSuffixLeadLabel == -1: "i=" + i + " lastSuffixLeadLabel=" + lastSuffixLeadLabel;
             suffixLeadLabel = -1;
           } else {
             suffixLeadLabel = term.termBytes[prefixLength] & 0xff;
@@ -587,10 +731,11 @@ public final class BlockTreeTermsWriter
             // block as soon as we have at least minItemsInBlock.  This is not always best: it often produces
             // a too-small block as the final block:
             boolean isFloor = itemsInBlock < count;
-            newBlocks.add(writeBlock(prefixLength, isFloor, nextFloorLeadLabel, nextBlockStart, i, hasTerms, hasSubBlocks));
+            newBlocks.add(writeBlock(prefixLength, isFloor, nextFloorLeadLabel, nextBlockStart, i, hasTerms, hasPrefixTerms, hasSubBlocks));
 
             hasTerms = false;
             hasSubBlocks = false;
+            hasPrefixTerms = false;
             nextFloorLeadLabel = suffixLeadLabel;
             nextBlockStart = i;
           }
@@ -600,6 +745,7 @@ public final class BlockTreeTermsWriter
 
         if (ent.isTerm) {
           hasTerms = true;
+          hasPrefixTerms |= ((PendingTerm) ent).prefixTerm != null;
         } else {
           hasSubBlocks = true;
         }
@@ -609,7 +755,7 @@ public final class BlockTreeTermsWriter
       if (nextBlockStart < end) {
         int itemsInBlock = end - nextBlockStart;
         boolean isFloor = itemsInBlock < count;
-        newBlocks.add(writeBlock(prefixLength, isFloor, nextFloorLeadLabel, nextBlockStart, end, hasTerms, hasSubBlocks));
+        newBlocks.add(writeBlock(prefixLength, isFloor, nextFloorLeadLabel, nextBlockStart, end, hasTerms, hasPrefixTerms, hasSubBlocks));
       }
 
       assert newBlocks.isEmpty() == false;
@@ -634,7 +780,8 @@ public final class BlockTreeTermsWriter
      *  were too many (more than maxItemsInBlock) entries sharing the
      *  same prefix, and so we broke it into multiple floor blocks where
      *  we record the starting label of the suffix of each floor block. */
-    private PendingBlock writeBlock(int prefixLength, boolean isFloor, int floorLeadLabel, int start, int end, boolean hasTerms, boolean hasSubBlocks) throws IOException {
+    private PendingBlock writeBlock(int prefixLength, boolean isFloor, int floorLeadLabel, int start, int end,
+                                    boolean hasTerms, boolean hasPrefixTerms, boolean hasSubBlocks) throws IOException {
 
       assert end > start;
 
@@ -646,6 +793,8 @@ public final class BlockTreeTermsWriter
       System.arraycopy(lastTerm.get().bytes, 0, prefix.bytes, 0, prefixLength);
       prefix.length = prefixLength;
 
+      //if (DEBUG2) System.out.println("    writeBlock field=" + fieldInfo.name + " prefix=" + brToString(prefix) + " fp=" + startFP + " isFloor=" + isFloor + " isLastInFloor=" + (end == pending.size()) + " floorLeadLabel=" + floorLeadLabel + " start=" + start + " end=" + end + " hasTerms=" + hasTerms + " hasSubBlocks=" + hasSubBlocks);
+
       // Write block header:
       int numEntries = end - start;
       int code = numEntries << 1;
@@ -666,31 +815,34 @@ public final class BlockTreeTermsWriter
 
       // We optimize the leaf block case (block has only terms), writing a more
       // compact format in this case:
-      boolean isLeafBlock = hasSubBlocks == false;
+      boolean isLeafBlock = hasSubBlocks == false && hasPrefixTerms == false;
+
+      //System.out.println("  isLeaf=" + isLeafBlock);
 
       final List<FST<BytesRef>> subIndices;
 
       boolean absolute = true;
 
       if (isLeafBlock) {
-        // Only terms:
+        // Block contains only ordinary terms:
         subIndices = null;
         for (int i=start;i<end;i++) {
           PendingEntry ent = pending.get(i);
           assert ent.isTerm: "i=" + i;
 
           PendingTerm term = (PendingTerm) ent;
+          assert term.prefixTerm == null;
+
           assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
           BlockTermState state = term.state;
           final int suffix = term.termBytes.length - prefixLength;
-          /*
-          if (DEBUG) {
-            BytesRef suffixBytes = new BytesRef(suffix);
-            System.arraycopy(term.termBytes, prefixLength, suffixBytes.bytes, 0, suffix);
-            suffixBytes.length = suffix;
-            System.out.println("    write term suffix=" + brToString(suffixBytes));
-          }
-          */
+          //if (DEBUG2) {
+          //  BytesRef suffixBytes = new BytesRef(suffix);
+          //  System.arraycopy(term.termBytes, prefixLength, suffixBytes.bytes, 0, suffix);
+          //  suffixBytes.length = suffix;
+          //  System.out.println("    write term suffix=" + brToString(suffixBytes));
+          //}
+
           // For leaf block we write suffix straight
           suffixWriter.writeVInt(suffix);
           suffixWriter.writeBytes(term.termBytes, prefixLength, suffix);
@@ -714,27 +866,51 @@ public final class BlockTreeTermsWriter
           absolute = false;
         }
       } else {
-        // Mixed terms and sub-blocks:
+        // Block has at least one prefix term or a sub block:
         subIndices = new ArrayList<>();
+        boolean sawAutoPrefixTerm = false;
         for (int i=start;i<end;i++) {
           PendingEntry ent = pending.get(i);
           if (ent.isTerm) {
             PendingTerm term = (PendingTerm) ent;
+
             assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
             BlockTermState state = term.state;
             final int suffix = term.termBytes.length - prefixLength;
-            /*
-            if (DEBUG) {
-              BytesRef suffixBytes = new BytesRef(suffix);
-              System.arraycopy(term.termBytes, prefixLength, suffixBytes.bytes, 0, suffix);
-              suffixBytes.length = suffix;
-              System.out.println("    write term suffix=" + brToString(suffixBytes));
-            }
-            */
+            //if (DEBUG2) {
+            //  BytesRef suffixBytes = new BytesRef(suffix);
+            //  System.arraycopy(term.termBytes, prefixLength, suffixBytes.bytes, 0, suffix);
+            //  suffixBytes.length = suffix;
+            //  System.out.println("      write term suffix=" + brToString(suffixBytes));
+            //  if (term.prefixTerm != null) {
+            //    System.out.println("        ** auto-prefix term: " + term.prefixTerm);
+            //  }
+            //}
+
             // For non-leaf block we borrow 1 bit to record
-            // if entry is term or sub-block
-            suffixWriter.writeVInt(suffix<<1);
+            // if entry is term or sub-block, and 1 bit to record if
+            // it's a prefix term.  Terms cannot be larger than ~32 KB
+            // so we won't run out of bits:
+            code = suffix<<2;
+            int floorLeadEnd = -1;
+            if (term.prefixTerm != null) {
+              sawAutoPrefixTerm = true;
+              PrefixTerm prefixTerm = term.prefixTerm;
+              floorLeadEnd = prefixTerm.floorLeadEnd;
+              assert floorLeadEnd != -1;
+
+              if (prefixTerm.floorLeadStart == -2) {
+                // Starts with empty string
+                code |= 2;
+              } else {
+                code |= 3;
+              }
+            }
+            suffixWriter.writeVInt(code);
             suffixWriter.writeBytes(term.termBytes, prefixLength, suffix);
+            if (floorLeadEnd != -1) {
+              suffixWriter.writeByte((byte) floorLeadEnd);
+            }
             assert floorLeadLabel == -1 || (term.termBytes[prefixLength] & 0xff) >= floorLeadLabel;
 
             // Write term stats, to separate byte[] blob:
@@ -765,33 +941,32 @@ public final class BlockTreeTermsWriter
             PendingBlock block = (PendingBlock) ent;
             assert StringHelper.startsWith(block.prefix, prefix);
             final int suffix = block.prefix.length - prefixLength;
+            assert StringHelper.startsWith(block.prefix, prefix);
 
             assert suffix > 0;
 
             // For non-leaf block we borrow 1 bit to record
-            // if entry is term or sub-block
-            suffixWriter.writeVInt((suffix<<1)|1);
+            // if entry is term or sub-block, and 1 bit (unset here) to
+            // record if it's a prefix term:
+            suffixWriter.writeVInt((suffix<<2)|1);
             suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix);
 
-            assert floorLeadLabel == -1 || (block.prefix.bytes[prefixLength] & 0xff) >= floorLeadLabel;
+            //if (DEBUG2) {
+            //  BytesRef suffixBytes = new BytesRef(suffix);
+            //  System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
+            //  suffixBytes.length = suffix;
+            //  System.out.println("      write sub-block suffix=" + brToString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor);
+            //}
 
+            assert floorLeadLabel == -1 || (block.prefix.bytes[prefixLength] & 0xff) >= floorLeadLabel: "floorLeadLabel=" + floorLeadLabel + " suffixLead=" + (block.prefix.bytes[prefixLength] & 0xff);
             assert block.fp < startFP;
 
-            /*
-            if (DEBUG) {
-              BytesRef suffixBytes = new BytesRef(suffix);
-              System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
-              suffixBytes.length = suffix;
-              System.out.println("    write sub-block suffix=" + brToString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor);
-            }
-            */
-
             suffixWriter.writeVLong(startFP - block.fp);
             subIndices.add(block.index);
           }
         }
 
-        assert subIndices.size() != 0;
+        assert subIndices.size() != 0 || sawAutoPrefixTerm;
       }
 
       // TODO: we could block-write the term suffix pointers;
@@ -835,7 +1010,7 @@ public final class BlockTreeTermsWriter
     }
     
     /** Writes one term's worth of postings. */
-    public void write(BytesRef text, TermsEnum termsEnum) throws IOException {
+    public void write(BytesRef text, TermsEnum termsEnum, PrefixTerm prefixTerm) throws IOException {
       /*
       if (DEBUG) {
         int[] tmp = new int[lastTerm.length];
@@ -846,19 +1021,25 @@ public final class BlockTreeTermsWriter
 
       BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen);
       if (state != null) {
+
         assert state.docFreq != 0;
         assert fieldInfo.getIndexOptions() == IndexOptions.DOCS || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter;
-        sumDocFreq += state.docFreq;
-        sumTotalTermFreq += state.totalTermFreq;
         pushTerm(text);
        
-        PendingTerm term = new PendingTerm(text, state);
+        PendingTerm term = new PendingTerm(text, state, prefixTerm);
         pending.add(term);
-        numTerms++;
-        if (firstPendingTerm == null) {
-          firstPendingTerm = term;
+        //if (DEBUG) System.out.println("    add pending term = " + text + " pending.size()=" + pending.size());
+
+        if (prefixTerm == null) {
+          // Only increment stats for real terms:
+          sumDocFreq += state.docFreq;
+          sumTotalTermFreq += state.totalTermFreq;
+          numTerms++;
+          if (firstPendingTerm == null) {
+            firstPendingTerm = term;
+          }
+          lastPendingTerm = term;
         }
-        lastPendingTerm = term;
       }
     }
 
@@ -910,6 +1091,7 @@ public final class BlockTreeTermsWriter
         // TODO: if pending.size() is already 1 with a non-zero prefix length
         // we can save writing a "degenerate" root block, but we have to
         // fix all the places that assume the root block's prefix is the empty string:
+        pushTerm(new BytesRef());
         writeBlocks(0, pending.size());
 
         // We better have one final "root" block:

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/FieldReader.java Thu Apr  2 15:05:48 2015
@@ -41,6 +41,8 @@ import org.apache.lucene.util.fst.FST;
  */
 public final class FieldReader extends Terms implements Accountable {
 
+  // private final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
   private static final long BASE_RAM_BYTES_USED =
       RamUsageEstimator.shallowSizeOfInstance(FieldReader.class)
       + 3 * RamUsageEstimator.shallowSizeOfInstance(BytesRef.class);
@@ -125,6 +127,7 @@ public final class FieldReader extends T
   /** For debugging -- used by CheckIndex too*/
   @Override
   public Stats getStats() throws IOException {
+    // TODO: add auto-prefix terms into stats
     return new SegmentTermsEnum(this).computeBlockStats();
   }
 
@@ -175,10 +178,11 @@ public final class FieldReader extends T
 
   @Override
   public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
-    if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
-      throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
-    }
-    return new IntersectTermsEnum(this, compiled, startTerm);
+    // if (DEBUG) System.out.println("  FieldReader.intersect startTerm=" + BlockTreeTermsWriter.brToString(startTerm));
+    //System.out.println("intersect: " + compiled.type + " a=" + compiled.automaton);
+    // TODO: we could push "it's a range" or "it's a prefix" down into IntersectTermsEnum?
+    // can we optimize knowing that...?
+    return new IntersectTermsEnum(this, compiled.automaton, compiled.runAutomaton, compiled.commonSuffixRef, startTerm, compiled.sinkState);
   }
     
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java Thu Apr  2 15:05:48 2015
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.PostingsEnum;
 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;
@@ -28,23 +29,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;
 
@@ -52,19 +68,34 @@ 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++) {
@@ -152,7 +183,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
@@ -220,7 +251,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);
@@ -229,23 +260,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!");
@@ -253,9 +290,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;
@@ -266,19 +305,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
@@ -297,77 +341,249 @@ 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;
 
@@ -381,24 +597,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;
           }
         }
@@ -410,10 +622,19 @@ 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);
@@ -423,16 +644,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);
@@ -440,6 +704,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/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnumFrame.java Thu Apr  2 15:05:48 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/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java?rev=1670918&r1=1670917&r2=1670918&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java Thu Apr  2 15:05:48 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();
       }
     }