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/07/24 20:23:07 UTC

svn commit: r1613235 [2/3] - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/analysis/ lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/util/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/ lucene...

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java?rev=1613235&r1=1613234&r2=1613235&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java Thu Jul 24 18:23:06 2014
@@ -45,11 +45,11 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.fst.NoOutputs;
 import org.apache.lucene.util.fst.Outputs;
 import org.apache.lucene.util.fst.Util;
 import org.apache.lucene.util.packed.PackedInts;
@@ -208,7 +208,7 @@ 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 final static boolean DEBUG = false;
   //private final static boolean SAVE_DOT_FILES = false;
 
   static final int OUTPUT_FLAGS_NUM_BITS = 2;
@@ -282,6 +282,7 @@ public final class BlockTreeTermsWriter 
   }
 
   private final List<FieldMetaData> fields = new ArrayList<>();
+
   // private final String segment;
 
   /** Create a new writer.  The number of items (terms or
@@ -328,7 +329,7 @@ public final class BlockTreeTermsWriter 
 
       currentField = null;
       this.postingsWriter = postingsWriter;
-      // segment = state.segmentName;
+      // segment = state.segmentInfo.name;
 
       // System.out.println("BTW.init seg=" + state.segmentName);
 
@@ -384,22 +385,42 @@ public final class BlockTreeTermsWriter 
   }
 
   private static final class PendingTerm extends PendingEntry {
-    public final BytesRef term;
+    public final byte[] termBytes;
     // stats + metadata
     public final BlockTermState state;
 
     public PendingTerm(BytesRef term, BlockTermState state) {
       super(true);
-      this.term = term;
+      this.termBytes = new byte[term.length];
+      System.arraycopy(term.bytes, term.offset, termBytes, 0, term.length);
       this.state = state;
     }
 
     @Override
     public String toString() {
-      return term.utf8ToString();
+      return 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();
     }
   }
 
+  // for debugging
+  @SuppressWarnings("unused")
+  static String brToString(byte[] b) {
+    return brToString(new BytesRef(b));
+  }
+
   private static final class PendingBlock extends PendingEntry {
     public final BytesRef prefix;
     public final long fp;
@@ -408,7 +429,6 @@ public final class BlockTreeTermsWriter 
     public final boolean hasTerms;
     public final boolean isFloor;
     public final int floorLeadByte;
-    private final IntsRef scratchIntsRef = new IntsRef();
 
     public PendingBlock(BytesRef prefix, long fp, boolean hasTerms, boolean isFloor, int floorLeadByte, List<FST<BytesRef>> subIndices) {
       super(false);
@@ -422,12 +442,13 @@ public final class BlockTreeTermsWriter 
 
     @Override
     public String toString() {
-      return "BLOCK: " + prefix.utf8ToString();
+      return "BLOCK: " + brToString(prefix);
     }
 
-    public void compileIndex(List<PendingBlock> floorBlocks, RAMOutputStream scratchBytes) throws IOException {
+    public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRef scratchIntsRef) throws IOException {
 
-      assert (isFloor && floorBlocks != null && floorBlocks.size() != 0) || (!isFloor && floorBlocks == null): "isFloor=" + isFloor + " floorBlocks=" + floorBlocks;
+      assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1): "isFloor=" + isFloor + " blocks=" + blocks;
+      assert this == blocks.get(0);
 
       assert scratchBytes.getFilePointer() == 0;
 
@@ -436,8 +457,9 @@ public final class BlockTreeTermsWriter 
       // outputs sharing in the FST
       scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor));
       if (isFloor) {
-        scratchBytes.writeVInt(floorBlocks.size());
-        for (PendingBlock sub : floorBlocks) {
+        scratchBytes.writeVInt(blocks.size()-1);
+        for (int i=1;i<blocks.size();i++) {
+          PendingBlock sub = blocks.get(i);
           assert sub.floorLeadByte != -1;
           //if (DEBUG) {
           //  System.out.println("    write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff));
@@ -450,9 +472,9 @@ public final class BlockTreeTermsWriter 
 
       final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
       final Builder<BytesRef> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1,
-                                                                   0, 0, true, false, Integer.MAX_VALUE,
-                                                                   outputs, null, false,
-                                                                   PackedInts.COMPACT, true, 15);
+                                                           0, 0, true, false, Integer.MAX_VALUE,
+                                                           outputs, false,
+                                                           PackedInts.COMPACT, true, 15);
       //if (DEBUG) {
       //  System.out.println("  compile index for prefix=" + prefix);
       //}
@@ -464,26 +486,18 @@ public final class BlockTreeTermsWriter 
       scratchBytes.reset();
 
       // Copy over index for all sub-blocks
-
-      if (subIndices != null) {
-        for(FST<BytesRef> subIndex : subIndices) {
-          append(indexBuilder, subIndex);
-        }
-      }
-
-      if (floorBlocks != null) {
-        for (PendingBlock sub : floorBlocks) {
-          if (sub.subIndices != null) {
-            for(FST<BytesRef> subIndex : sub.subIndices) {
-              append(indexBuilder, subIndex);
-            }
+      for(PendingBlock block : blocks) {
+        if (block.subIndices != null) {
+          for(FST<BytesRef> subIndex : block.subIndices) {
+            append(indexBuilder, subIndex, scratchIntsRef);
           }
-          sub.subIndices = null;
+          block.subIndices = null;
         }
       }
 
       index = indexBuilder.finish();
-      subIndices = null;
+
+      assert subIndices == null;
 
       /*
       Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
@@ -496,7 +510,7 @@ public final class BlockTreeTermsWriter 
     // TODO: maybe we could add bulk-add method to
     // Builder?  Takes FST and unions it w/ current
     // FST.
-    private void append(Builder<BytesRef> builder, FST<BytesRef> subIndex) throws IOException {
+    private void append(Builder<BytesRef> builder, FST<BytesRef> subIndex, IntsRef scratchIntsRef) throws IOException {
       final BytesRefFSTEnum<BytesRef> subIndexEnum = new BytesRefFSTEnum<>(subIndex);
       BytesRefFSTEnum.InputOutput<BytesRef> indexEnt;
       while((indexEnt = subIndexEnum.next()) != null) {
@@ -508,7 +522,8 @@ public final class BlockTreeTermsWriter 
     }
   }
 
-  final RAMOutputStream scratchBytes = new RAMOutputStream();
+  private final RAMOutputStream scratchBytes = new RAMOutputStream();
+  private final IntsRef scratchIntsRef = new IntsRef();
 
   class TermsWriter extends TermsConsumer {
     private final FieldInfo fieldInfo;
@@ -520,384 +535,199 @@ public final class BlockTreeTermsWriter 
     int docCount;
     long indexStartFP;
 
-    // Used only to partition terms into the block tree; we
-    // don't pull an FST from this builder:
-    private final NoOutputs noOutputs;
-    private final Builder<Object> blockBuilder;
-
-    // PendingTerm or PendingBlock:
+    // Records index into pending where the current prefix at that
+    // length "started"; for example, if current term starts with 't',
+    // startsByPrefix[0] is the index into pending for the first
+    // term/sub-block starting with 't'.  We use this to figure out when
+    // to write a new block:
+    private final BytesRef lastTerm = new BytesRef();
+    private int[] prefixStarts = new int[8];
+
+    private final long[] longs;
+
+    // Pending stack of terms and blocks.  As terms arrive (in sorted order)
+    // we append to this stack, and once the top of the stack has enough
+    // terms starting with a common prefix, write write a new block with
+    // those terms and replace those terms in the stack with a new block:
     private final List<PendingEntry> pending = new ArrayList<>();
 
-    // Index into pending of most recently written block
-    private int lastBlockIndex = -1;
-
-    // Re-used when segmenting a too-large block into floor
-    // blocks:
-    private int[] subBytes = new int[10];
-    private int[] subTermCounts = new int[10];
-    private int[] subTermCountSums = new int[10];
-    private int[] subSubCounts = new int[10];
+    // Reused in writeBlocks:
+    private final List<PendingBlock> newBlocks = new ArrayList<>();
 
     private BytesRef minTerm;
     private BytesRef maxTerm = new BytesRef();
 
-    // This class assigns terms to blocks "naturally", ie,
-    // according to the number of terms under a given prefix
-    // that we encounter:
-    private class FindBlocks extends Builder.FreezeTail<Object> {
+    /** Writes the top count entries in pending, using prevTerm to compute the prefix. */
+    void writeBlocks(int prefixLength, int count) throws IOException {
 
-      @Override
-      public void freeze(final Builder.UnCompiledNode<Object>[] frontier, int prefixLenPlus1, final IntsRef lastInput) throws IOException {
+      assert count > 0;
 
-        //if (DEBUG) System.out.println("  freeze prefixLenPlus1=" + prefixLenPlus1);
-
-        for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) {
-          final Builder.UnCompiledNode<Object> node = frontier[idx];
-
-          long totCount = 0;
-
-          if (node.isFinal) {
-            totCount++;
-          }
-
-          for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
-            @SuppressWarnings("unchecked") final Builder.UnCompiledNode<Object> target = (Builder.UnCompiledNode<Object>) node.arcs[arcIdx].target;
-            totCount += target.inputCount;
-            target.clear();
-            node.arcs[arcIdx].target = null;
-          }
-          node.numArcs = 0;
-
-          if (totCount >= minItemsInBlock || idx == 0) {
-            // We are on a prefix node that has enough
-            // entries (terms or sub-blocks) under it to let
-            // us write a new block or multiple blocks (main
-            // block + follow on floor blocks):
-            //if (DEBUG) {
-            //  if (totCount < minItemsInBlock && idx != 0) {
-            //    System.out.println("  force block has terms");
-            //  }
-            //}
-            writeBlocks(lastInput, idx, (int) totCount);
-            node.inputCount = 1;
-          } else {
-            // stragglers!  carry count upwards
-            node.inputCount = totCount;
-          }
-          frontier[idx] = new Builder.UnCompiledNode<>(blockBuilder, idx);
-        }
+      /*
+      if (DEBUG) {
+        BytesRef br = new BytesRef(lastTerm.bytes);
+        br.offset = lastTerm.offset;
+        br.length = prefixLength;
+        System.out.println("writeBlocks: " + br.utf8ToString() + " count=" + count);
       }
-    }
-
-    // Write the top count entries on the pending stack as
-    // one or more blocks.  If the entry count is <= maxItemsPerBlock
-    // we just write a single block; else we break into
-    // primary (initial) block and then one or more
-    // following floor blocks:
-
-    void writeBlocks(IntsRef prevTerm, int prefixLength, int count) throws IOException {
-      // System.out.println("writeBlocks count=" + count);
-      if (count <= maxItemsInBlock) {
-        // Easy case: not floor block.  Eg, prefix is "foo",
-        // and we found 30 terms/sub-blocks starting w/ that
-        // prefix, and minItemsInBlock <= 30 <=
-        // maxItemsInBlock.
-        final PendingBlock nonFloorBlock = writeBlock(prevTerm, prefixLength, prefixLength, count, count, 0, false, -1, true);
-        nonFloorBlock.compileIndex(null, scratchBytes);
-        pending.add(nonFloorBlock);
-        // System.out.println("  1 block");
-      } else {
-        // Floor block case.  Eg, prefix is "foo" but we
-        // have 100 terms/sub-blocks starting w/ that
-        // prefix.  We segment the entries into a primary
-        // block and following floor blocks using the first
-        // label in the suffix to assign to floor blocks.
+      */
 
-        // TODO: we could store min & max suffix start byte
-        // in each block, to make floor blocks authoritative
+      // Root block better write all remaining pending entries:
+      assert prefixLength > 0 || count == pending.size();
 
-        //if (DEBUG) {
-        //  final BytesRef prefix = new BytesRef(prefixLength);
-        //  for(int m=0;m<prefixLength;m++) {
-        //    prefix.bytes[m] = (byte) prevTerm.ints[m];
-        //  }
-        //  prefix.length = prefixLength;
-        //  //System.out.println("\nWBS count=" + count + " prefix=" + prefix.utf8ToString() + " " + prefix);
-        //  System.out.println("writeBlocks: prefix=" + prefix + " " + prefix + " count=" + count + " pending.size()=" + pending.size());
-        //}
-        //System.out.println("\nwbs count=" + count);
+      int lastSuffixLeadLabel = -1;
 
-        final int savLabel = prevTerm.ints[prevTerm.offset + prefixLength];
+      // True if we saw at least one term in this block (we record if a block
+      // 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 hasSubBlocks = false;
 
-        // Count up how many items fall under
-        // each unique label after the prefix.
-        
-        // TODO: this is wasteful since the builder had
-        // already done this (partitioned these sub-terms
-        // according to their leading prefix byte)
-        
-        final List<PendingEntry> slice = pending.subList(pending.size()-count, pending.size());
-        int lastSuffixLeadLabel = -1;
-        int termCount = 0;
-        int subCount = 0;
-        int numSubs = 0;
+      int start = pending.size()-count;
+      int end = pending.size();
+      int nextBlockStart = start;
+      int nextFloorLeadLabel = -1;
 
-        for(PendingEntry ent : slice) {
+      for (int i=start; i<end; i++) {
 
-          // First byte in the suffix of this term
-          final int suffixLeadLabel;
-          if (ent.isTerm) {
-            PendingTerm term = (PendingTerm) ent;
-            if (term.term.length == prefixLength) {
-              // Suffix is 0, ie prefix 'foo' and term is
-              // 'foo' so the term has empty string suffix
-              // in this block
-              assert lastSuffixLeadLabel == -1;
-              assert numSubs == 0;
-              suffixLeadLabel = -1;
-            } else {
-              suffixLeadLabel = term.term.bytes[term.term.offset + prefixLength] & 0xff;
-            }
-          } else {
-            PendingBlock block = (PendingBlock) ent;
-            assert block.prefix.length > prefixLength;
-            suffixLeadLabel = block.prefix.bytes[block.prefix.offset + prefixLength] & 0xff;
-          }
+        PendingEntry ent = pending.get(i);
 
-          if (suffixLeadLabel != lastSuffixLeadLabel && (termCount + subCount) != 0) {
-            if (subBytes.length == numSubs) {
-              subBytes = ArrayUtil.grow(subBytes);
-              subTermCounts = ArrayUtil.grow(subTermCounts);
-              subSubCounts = ArrayUtil.grow(subSubCounts);
-            }
-            subBytes[numSubs] = lastSuffixLeadLabel;
-            lastSuffixLeadLabel = suffixLeadLabel;
-            subTermCounts[numSubs] = termCount;
-            subSubCounts[numSubs] = subCount;
-            /*
-            if (suffixLeadLabel == -1) {
-              System.out.println("  sub " + -1 + " termCount=" + termCount + " subCount=" + subCount);
-            } else {
-              System.out.println("  sub " + Integer.toHexString(suffixLeadLabel) + " termCount=" + termCount + " subCount=" + subCount);
-            }
-            */
-            termCount = subCount = 0;
-            numSubs++;
-          }
+        int suffixLeadLabel;
 
-          if (ent.isTerm) {
-            termCount++;
+        if (ent.isTerm) {
+          PendingTerm term = (PendingTerm) ent;
+          if (term.termBytes.length == prefixLength) {
+            // Suffix is 0, i.e. prefix 'foo' and term is
+            // 'foo' so the term has empty string suffix
+            // in this block
+            assert lastSuffixLeadLabel == -1;
+            suffixLeadLabel = -1;
           } else {
-            subCount++;
+            suffixLeadLabel = term.termBytes[prefixLength] & 0xff;
+          }
+        } else {
+          PendingBlock block = (PendingBlock) ent;
+          assert block.prefix.length > prefixLength;
+          suffixLeadLabel = block.prefix.bytes[block.prefix.offset + prefixLength] & 0xff;
+        }
+        // if (DEBUG) System.out.println("  i=" + i + " ent=" + ent + " suffixLeadLabel=" + suffixLeadLabel);
+
+        if (suffixLeadLabel != lastSuffixLeadLabel) {
+          int itemsInBlock = i - nextBlockStart;
+          if (itemsInBlock >= minItemsInBlock && end-nextBlockStart > maxItemsInBlock) {
+            // The count is too large for one block, so we must break it into "floor" blocks, where we record
+            // the leading label of the suffix of the first term in each floor block, so at search time we can
+            // jump to the right floor block.  We just use a naive greedy segmenter here: make a new floor
+            // 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));
+
+            hasTerms = false;
+            hasSubBlocks = false;
+            nextFloorLeadLabel = suffixLeadLabel;
+            nextBlockStart = i;
           }
-        }
 
-        if (subBytes.length == numSubs) {
-          subBytes = ArrayUtil.grow(subBytes);
-          subTermCounts = ArrayUtil.grow(subTermCounts);
-          subSubCounts = ArrayUtil.grow(subSubCounts);
+          lastSuffixLeadLabel = suffixLeadLabel;
         }
 
-        subBytes[numSubs] = lastSuffixLeadLabel;
-        subTermCounts[numSubs] = termCount;
-        subSubCounts[numSubs] = subCount;
-        numSubs++;
-        /*
-        if (lastSuffixLeadLabel == -1) {
-          System.out.println("  sub " + -1 + " termCount=" + termCount + " subCount=" + subCount);
+        if (ent.isTerm) {
+          hasTerms = true;
         } else {
-          System.out.println("  sub " + Integer.toHexString(lastSuffixLeadLabel) + " termCount=" + termCount + " subCount=" + subCount);
-        }
-        */
-
-        if (subTermCountSums.length < numSubs) {
-          subTermCountSums = ArrayUtil.grow(subTermCountSums, numSubs);
+          hasSubBlocks = true;
         }
+      }
 
-        // Roll up (backwards) the termCounts; postings impl
-        // needs this to know where to pull the term slice
-        // from its pending terms stack:
-        int sum = 0;
-        for(int idx=numSubs-1;idx>=0;idx--) {
-          sum += subTermCounts[idx];
-          subTermCountSums[idx] = sum;
-        }
+      // Write last block, if any:
+      if (nextBlockStart < end) {
+        int itemsInBlock = end - nextBlockStart;
+        boolean isFloor = itemsInBlock < count;
+        newBlocks.add(writeBlock(prefixLength, isFloor, nextFloorLeadLabel, nextBlockStart, end, hasTerms, hasSubBlocks));
+      }
 
-        // TODO: make a better segmenter?  It'd have to
-        // absorb the too-small end blocks backwards into
-        // the previous blocks
-
-        // Naive greedy segmentation; this is not always
-        // best (it can produce a too-small block as the
-        // last block):
-        int pendingCount = 0;
-        int startLabel = subBytes[0];
-        int curStart = count;
-        subCount = 0;
-
-        final List<PendingBlock> floorBlocks = new ArrayList<>();
-        PendingBlock firstBlock = null;
-
-        for(int sub=0;sub<numSubs;sub++) {
-          pendingCount += subTermCounts[sub] + subSubCounts[sub];
-          //System.out.println("  " + (subTermCounts[sub] + subSubCounts[sub]));
-          subCount++;
-
-          // Greedily make a floor block as soon as we've
-          // crossed the min count
-          if (pendingCount >= minItemsInBlock) {
-            final int curPrefixLength;
-            if (startLabel == -1) {
-              curPrefixLength = prefixLength;
-            } else {
-              curPrefixLength = 1+prefixLength;
-              // floor term:
-              prevTerm.ints[prevTerm.offset + prefixLength] = startLabel;
-            }
-            //System.out.println("  " + subCount + " subs");
-            final PendingBlock floorBlock = writeBlock(prevTerm, prefixLength, curPrefixLength, curStart, pendingCount, subTermCountSums[1+sub], true, startLabel, curStart == pendingCount);
-            if (firstBlock == null) {
-              firstBlock = floorBlock;
-            } else {
-              floorBlocks.add(floorBlock);
-            }
-            curStart -= pendingCount;
-            // System.out.println("  floor=" + pendingCount);
-            //System.out.println("    = " + pendingCount);
-            pendingCount = 0;
-
-            assert minItemsInBlock == 1 || subCount > 1: "minItemsInBlock=" + minItemsInBlock + " subCount=" + subCount + " sub=" + sub + " of " + numSubs + " subTermCount=" + subTermCountSums[sub] + " subSubCount=" + subSubCounts[sub] + " depth=" + prefixLength;
-            subCount = 0;
-            startLabel = subBytes[sub+1];
+      assert newBlocks.isEmpty() == false;
 
-            if (curStart == 0) {
-              break;
-            }
+      PendingBlock firstBlock = newBlocks.get(0);
 
-            if (curStart <= maxItemsInBlock) {
-              // remainder is small enough to fit into a
-              // block.  NOTE that this may be too small (<
-              // minItemsInBlock); need a true segmenter
-              // here
-              assert startLabel != -1;
-              assert firstBlock != null;
-              prevTerm.ints[prevTerm.offset + prefixLength] = startLabel;
-              //System.out.println("  final " + (numSubs-sub-1) + " subs");
-              /*
-              for(sub++;sub < numSubs;sub++) {
-                System.out.println("  " + (subTermCounts[sub] + subSubCounts[sub]));
-              }
-              System.out.println("    = " + curStart);
-              if (curStart < minItemsInBlock) {
-                System.out.println("      **");
-              }
-              */
-              floorBlocks.add(writeBlock(prevTerm, prefixLength, prefixLength+1, curStart, curStart, 0, true, startLabel, true));
-              break;
-            }
-          }
-        }
+      assert firstBlock.isFloor || newBlocks.size() == 1;
 
-        prevTerm.ints[prevTerm.offset + prefixLength] = savLabel;
+      firstBlock.compileIndex(newBlocks, scratchBytes, scratchIntsRef);
 
-        assert firstBlock != null;
-        firstBlock.compileIndex(floorBlocks, scratchBytes);
+      // Remove slice from the top of the pending stack, that we just wrote:
+      pending.subList(pending.size()-count, pending.size()).clear();
 
-        pending.add(firstBlock);
-        //if (DEBUG) System.out.println("  done pending.size()=" + pending.size());
-      }
-      lastBlockIndex = pending.size()-1;
-    }
+      // Append new block
+      pending.add(firstBlock);
 
-    // for debugging
-    @SuppressWarnings("unused")
-    private String toString(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();
-      }
+      newBlocks.clear();
     }
 
-    // Writes all entries in the pending slice as a single
-    // block: 
-    private PendingBlock writeBlock(IntsRef prevTerm, int prefixLength, int indexPrefixLength, int startBackwards, int length,
-                                    int futureTermCount, boolean isFloor, int floorLeadByte, boolean isLastInFloor) throws IOException {
+    /** Writes the specified slice (start is inclusive, end is exclusive)
+     *  from pending stack as a new block.  If isFloor is true, there
+     *  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 {
 
-      assert length > 0;
+      assert end > start;
 
-      final int start = pending.size()-startBackwards;
+      long startFP = out.getFilePointer();
 
-      assert start >= 0: "pending.size()=" + pending.size() + " startBackwards=" + startBackwards + " length=" + length;
+      // if (DEBUG) System.out.println("    writeBlock fp=" + startFP + " isFloor=" + isFloor + " floorLeadLabel=" + floorLeadLabel + " start=" + start + " end=" + end + " hasTerms=" + hasTerms + " hasSubBlocks=" + hasSubBlocks);
 
-      final List<PendingEntry> slice = pending.subList(start, start + length);
+      boolean hasFloorLeadLabel = isFloor && floorLeadLabel != -1;
 
-      final long startFP = out.getFilePointer();
-
-      final BytesRef prefix = new BytesRef(indexPrefixLength);
-      for(int m=0;m<indexPrefixLength;m++) {
-        prefix.bytes[m] = (byte) prevTerm.ints[m];
-      }
-      prefix.length = indexPrefixLength;
+      final BytesRef prefix = new BytesRef(prefixLength + (hasFloorLeadLabel ? 1 : 0));
+      System.arraycopy(lastTerm.bytes, 0, prefix.bytes, 0, prefixLength);
+      prefix.length = prefixLength;
 
       // Write block header:
-      out.writeVInt((length<<1)|(isLastInFloor ? 1:0));
+      int numEntries = end - start;
+      int code = numEntries << 1;
+      if (end == pending.size()) {
+        // Last block:
+        code |= 1;
+      }
+      out.writeVInt(code);
 
       // if (DEBUG) {
-      //   System.out.println("  writeBlock " + (isFloor ? "(floor) " : "") + "seg=" + segment + " pending.size()=" + pending.size() + " prefixLength=" + prefixLength + " indexPrefix=" + toString(prefix) + " entCount=" + length + " startFP=" + startFP + " futureTermCount=" + futureTermCount + (isFloor ? (" floorLeadByte=" + Integer.toHexString(floorLeadByte&0xff)) : "") + " isLastInFloor=" + isLastInFloor);
+      //   System.out.println("  writeBlock " + (isFloor ? "(floor) " : "") + "seg=" + segment + " pending.size()=" + pending.size() + " prefixLength=" + prefixLength + " indexPrefix=" + brToString(prefix) + " entCount=" + length + " startFP=" + startFP + (isFloor ? (" floorLeadByte=" + Integer.toHexString(floorLeadByte&0xff)) : "") + " isLastInFloor=" + isLastInFloor);
       // }
 
       // 1st pass: pack term suffix bytes into byte[] blob
       // TODO: cutover to bulk int codec... simple64?
 
-      final boolean isLeafBlock;
-      if (lastBlockIndex < start) {
-        // This block definitely does not contain sub-blocks:
-        isLeafBlock = true;
-        //System.out.println("no scan true isFloor=" + isFloor);
-      } else if (!isFloor) {
-        // This block definitely does contain at least one sub-block:
-        isLeafBlock = false;
-        //System.out.println("no scan false " + lastBlockIndex + " vs start=" + start + " len=" + length);
-      } else {
-        // Must scan up-front to see if there is a sub-block
-        boolean v = true;
-        //System.out.println("scan " + lastBlockIndex + " vs start=" + start + " len=" + length);
-        for (PendingEntry ent : slice) {
-          if (!ent.isTerm) {
-            v = false;
-            break;
-          }
-        }
-        isLeafBlock = v;
-      }
+      // We optimize the leaf block case (block has only terms), writing a more
+      // compact format in this case:
+      boolean isLeafBlock = hasSubBlocks == false;
 
       final List<FST<BytesRef>> subIndices;
 
-      int termCount;
-
-      long[] longs = new long[longsSize];
       boolean absolute = true;
 
       if (isLeafBlock) {
+        // Only terms:
         subIndices = null;
-        for (PendingEntry ent : slice) {
-          assert ent.isTerm;
+        for (int i=start;i<end;i++) {
+          PendingEntry ent = pending.get(i);
+          assert ent.isTerm: "i=" + i;
+
           PendingTerm term = (PendingTerm) ent;
+          assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
           BlockTermState state = term.state;
-          final int suffix = term.term.length - prefixLength;
-          // if (DEBUG) {
-          //   BytesRef suffixBytes = new BytesRef(suffix);
-          //   System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
-          //   suffixBytes.length = suffix;
-          //   System.out.println("    write term suffix=" + suffixBytes);
-          // }
+          final int suffix = term.termBytes.length - prefixLength;
+          /*
+          if (DEBUG) {
+            BytesRef suffixBytes = new BytesRef(suffix);
+            System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
+            suffixBytes.length = suffix;
+            System.out.println("    write term suffix=" + suffixBytes);
+          }
+          */
           // For leaf block we write suffix straight
           suffixWriter.writeVInt(suffix);
-          suffixWriter.writeBytes(term.term.bytes, prefixLength, suffix);
+          suffixWriter.writeBytes(term.termBytes, prefixLength, suffix);
+          assert floorLeadLabel == -1 || (term.termBytes[prefixLength] & 0xff) >= floorLeadLabel;
 
           // Write term stats, to separate byte[] blob:
           statsWriter.writeVInt(state.docFreq);
@@ -916,25 +746,29 @@ public final class BlockTreeTermsWriter 
           bytesWriter.reset();
           absolute = false;
         }
-        termCount = length;
       } else {
+        // Mixed terms and sub-blocks:
         subIndices = new ArrayList<>();
-        termCount = 0;
-        for (PendingEntry ent : slice) {
+        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.term.length - prefixLength;
-            // if (DEBUG) {
-            //   BytesRef suffixBytes = new BytesRef(suffix);
-            //   System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
-            //   suffixBytes.length = suffix;
-            //   System.out.println("    write term suffix=" + suffixBytes);
-            // }
+            final int suffix = term.termBytes.length - prefixLength;
+            /*
+            if (DEBUG) {
+              BytesRef suffixBytes = new BytesRef(suffix);
+              System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
+              suffixBytes.length = suffix;
+              System.out.println("    write term suffix=" + suffixBytes);
+            }
+            */
             // For non-leaf block we borrow 1 bit to record
             // if entry is term or sub-block
             suffixWriter.writeVInt(suffix<<1);
-            suffixWriter.writeBytes(term.term.bytes, prefixLength, suffix);
+            suffixWriter.writeBytes(term.termBytes, prefixLength, suffix);
+            assert floorLeadLabel == -1 || (term.termBytes[prefixLength] & 0xff) >= floorLeadLabel;
 
             // Write term stats, to separate byte[] blob:
             statsWriter.writeVInt(state.docFreq);
@@ -960,10 +794,9 @@ public final class BlockTreeTermsWriter 
             bytesWriter.writeTo(metaWriter);
             bytesWriter.reset();
             absolute = false;
-
-            termCount++;
           } else {
             PendingBlock block = (PendingBlock) ent;
+            assert StringHelper.startsWith(block.prefix, prefix);
             final int suffix = block.prefix.length - prefixLength;
 
             assert suffix > 0;
@@ -972,14 +805,19 @@ public final class BlockTreeTermsWriter 
             // if entry is term or sub-block
             suffixWriter.writeVInt((suffix<<1)|1);
             suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix);
+
+            assert floorLeadLabel == -1 || (block.prefix.bytes[prefixLength] & 0xff) >= floorLeadLabel;
+
             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=" + toString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor);
-            // }
+            /*
+            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);
@@ -1008,42 +846,24 @@ public final class BlockTreeTermsWriter 
       metaWriter.writeTo(out);
       metaWriter.reset();
 
-      // Remove slice replaced by block:
-      slice.clear();
-
-      if (lastBlockIndex >= start) {
-        if (lastBlockIndex < start+length) {
-          lastBlockIndex = start;
-        } else {
-          lastBlockIndex -= length;
-        }
-      }
-
       // if (DEBUG) {
       //   System.out.println("      fpEnd=" + out.getFilePointer());
       // }
 
-      return new PendingBlock(prefix, startFP, termCount != 0, isFloor, floorLeadByte, subIndices);
+      if (hasFloorLeadLabel) {
+        // We already allocated to length+1 above:
+        prefix.bytes[prefix.length++] = (byte) floorLeadLabel;
+      }
+
+      return new PendingBlock(prefix, startFP, hasTerms, isFloor, floorLeadLabel, subIndices);
     }
 
     TermsWriter(FieldInfo fieldInfo) {
       this.fieldInfo = fieldInfo;
       docsSeen = new FixedBitSet(maxDoc);
 
-      noOutputs = NoOutputs.getSingleton();
-
-      // This Builder is just used transiently to fragment
-      // terms into "good" blocks; we don't save the
-      // resulting FST:
-      blockBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1,
-                                   0, 0, true,
-                                   true, Integer.MAX_VALUE,
-                                   noOutputs,
-                                   new FindBlocks(), false,
-                                   PackedInts.COMPACT,
-                                   true, 15);
-
       this.longsSize = postingsWriter.setField(fieldInfo);
+      this.longs = new long[longsSize];
     }
     
     @Override
@@ -1065,34 +885,85 @@ public final class BlockTreeTermsWriter 
       return postingsWriter;
     }
 
-    private final IntsRef scratchIntsRef = new IntsRef();
-
     @Override
     public void finishTerm(BytesRef text, TermStats stats) throws IOException {
+      /*
+      if (DEBUG) {
+        int[] tmp = new int[lastTerm.length];
+        System.arraycopy(prefixStarts, 0, tmp, 0, tmp.length);
+        System.out.println("BTTW: write term=" + brToString(text) + " prefixStarts=" + Arrays.toString(tmp) + " pending.size()=" + pending.size());
+      }
+      */
 
       assert stats.docFreq > 0;
       //if (DEBUG) System.out.println("BTTW.finishTerm term=" + fieldInfo.name + ":" + toString(text) + " seg=" + segment + " df=" + stats.docFreq);
 
-      blockBuilder.add(Util.toIntsRef(text, scratchIntsRef), noOutputs.getNoOutput());
+      assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY || stats.totalTermFreq >= stats.docFreq: "postingsWriter=" + postingsWriter;
+
       BlockTermState state = postingsWriter.newTermState();
       state.docFreq = stats.docFreq;
       state.totalTermFreq = stats.totalTermFreq;
       postingsWriter.finishTerm(state);
 
-      PendingTerm term = new PendingTerm(BytesRef.deepCopyOf(text), state);
-      pending.add(term);
-      numTerms++;
-
       if (minTerm == null) {
         minTerm = BytesRef.deepCopyOf(text);
       }
+      sumDocFreq += state.docFreq;
+      sumTotalTermFreq += state.totalTermFreq;
+      pushTerm(text);
+       
+      PendingTerm term = new PendingTerm(text, state);
+      pending.add(term);
+      numTerms++;
       maxTerm.copyBytes(text);
     }
 
+    /** Pushes the new term to the top of the stack, and writes new blocks. */
+    private void pushTerm(BytesRef text) throws IOException {
+      int limit = Math.min(lastTerm.length, text.length);
+
+      // Find common prefix between last term and current term:
+      int pos = 0;
+      while (pos < limit && lastTerm.bytes[pos] == text.bytes[text.offset+pos]) {
+        pos++;
+      }
+
+      // if (DEBUG) System.out.println("  shared=" + pos + "  lastTerm.length=" + lastTerm.length);
+
+      // Close the "abandoned" suffix now:
+      for(int i=lastTerm.length-1;i>=pos;i--) {
+
+        // How many items on top of the stack share the current suffix
+        // we are closing:
+        int prefixTopSize = pending.size() - prefixStarts[i];
+        if (prefixTopSize >= minItemsInBlock) {
+          // if (DEBUG) System.out.println("pushTerm i=" + i + " prefixTopSize=" + prefixTopSize + " minItemsInBlock=" + minItemsInBlock);
+          writeBlocks(i+1, prefixTopSize);
+          prefixStarts[i] -= prefixTopSize-1;
+        }
+      }
+
+      if (prefixStarts.length < text.length) {
+        prefixStarts = ArrayUtil.grow(prefixStarts, text.length);
+      }
+
+      // Init new tail:
+      for(int i=pos;i<text.length;i++) {
+        prefixStarts[i] = pending.size();
+      }
+
+      lastTerm.copyBytes(text);
+    }
+
     @Override
     public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
       if (numTerms > 0) {
-        blockBuilder.finish();
+        // if (DEBUG) System.out.println("BTTW: finish prefixStarts=" + Arrays.toString(prefixStarts));
+
+        // 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:
+        writeBlocks(0, pending.size());
 
         // We better have one final "root" block:
         assert pending.size() == 1 && !pending.get(0).isTerm: "pending.size()=" + pending.size() + " pending=" + pending;
@@ -1109,13 +980,15 @@ public final class BlockTreeTermsWriter 
         root.index.save(indexOut);
         //System.out.println("  write FST " + indexStartFP + " field=" + fieldInfo.name);
 
-        // if (SAVE_DOT_FILES || DEBUG) {
-        //   final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
-        //   Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
-        //   Util.toDot(root.index, w, false, false);
-        //   System.out.println("SAVED to " + dotFileName);
-        //   w.close();
-        // }
+        /*
+        if (DEBUG) {
+          final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
+          Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
+          Util.toDot(root.index, w, false, false);
+          System.out.println("SAVED to " + dotFileName);
+          w.close();
+        }
+        */
 
         fields.add(new FieldMetaData(fieldInfo,
                                      ((PendingBlock) pending.get(0)).index.getEmptyOutput(),

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java?rev=1613235&r1=1613234&r2=1613235&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java Thu Jul 24 18:23:06 2014
@@ -153,7 +153,8 @@ final class SegmentTermsEnumFrame {
     entCount = code >>> 1;
     assert entCount > 0;
     isLastInFloor = (code & 1) != 0;
-    assert arc == null || (isLastInFloor || isFloor);
+
+    assert arc == null || (isLastInFloor || isFloor): "fp=" + fp + " arc=" + arc + " isFloor=" + isFloor + " isLastInFloor=" + isLastInFloor;
 
     // TODO: if suffixes were stored in random-access
     // array structure, then we could do binary search

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/StringHelper.java?rev=1613235&r1=1613234&r2=1613235&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/StringHelper.java Thu Jul 24 18:23:06 2014
@@ -102,6 +102,31 @@ public abstract class StringHelper {
    * Otherwise <code>false</code>.
    * 
    * @param ref
+   *         the {@code byte[]} to test
+   * @param prefix
+   *         the expected prefix
+   * @return Returns <code>true</code> iff the ref starts with the given prefix.
+   *         Otherwise <code>false</code>.
+   */
+  public static boolean startsWith(byte[] ref, BytesRef prefix) {
+    if (ref.length < prefix.length) {
+      return false;
+    }
+
+    for(int i=0;i<prefix.length;i++) {
+      if (ref[i] != prefix.bytes[prefix.offset+i]) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * Returns <code>true</code> iff the ref starts with the given prefix.
+   * Otherwise <code>false</code>.
+   * 
+   * @param ref
    *          the {@link BytesRef} to test
    * @param prefix
    *          the expected prefix

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java?rev=1613235&r1=1613234&r2=1613235&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java Thu Jul 24 18:23:06 2014
@@ -80,22 +80,14 @@ public class Builder<T> {
   // current "frontier"
   private UnCompiledNode<T>[] frontier;
 
-  /** Expert: this is invoked by Builder whenever a suffix
-   *  is serialized. */
-  public static abstract class FreezeTail<T> {
-    public abstract void freeze(final UnCompiledNode<T>[] frontier, int prefixLenPlus1, IntsRef prevInput) throws IOException;
-  }
-
-  private final FreezeTail<T> freezeTail;
-
   /**
    * Instantiates an FST/FSA builder without any pruning. A shortcut
    * to {@link #Builder(FST.INPUT_TYPE, int, int, boolean,
-   * boolean, int, Outputs, FreezeTail, boolean, float,
+   * boolean, int, Outputs, boolean, float,
    * boolean, int)} with pruning options turned off.
    */
   public Builder(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
-    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, false, PackedInts.COMPACT, true, 15);
+    this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, false, PackedInts.COMPACT, true, 15);
   }
 
   /**
@@ -151,11 +143,10 @@ public class Builder<T> {
    */
   public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
                  boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
-                 FreezeTail<T> freezeTail, boolean doPackFST, float acceptableOverheadRatio, boolean allowArrayArcs,
+                 boolean doPackFST, float acceptableOverheadRatio, boolean allowArrayArcs,
                  int bytesPageBits) {
     this.minSuffixCount1 = minSuffixCount1;
     this.minSuffixCount2 = minSuffixCount2;
-    this.freezeTail = freezeTail;
     this.doShareNonSingletonNodes = doShareNonSingletonNodes;
     this.shareMaxTailLength = shareMaxTailLength;
     this.doPackFST = doPackFST;
@@ -209,100 +200,95 @@ public class Builder<T> {
   }
 
   private void freezeTail(int prefixLenPlus1) throws IOException {
-    if (freezeTail != null) {
-      // Custom plugin:
-      freezeTail.freeze(frontier, prefixLenPlus1, lastInput);
-    } else {
-      //System.out.println("  compileTail " + prefixLenPlus1);
-      final int downTo = Math.max(1, prefixLenPlus1);
-      for(int idx=lastInput.length; idx >= downTo; idx--) {
+    //System.out.println("  compileTail " + prefixLenPlus1);
+    final int downTo = Math.max(1, prefixLenPlus1);
+    for(int idx=lastInput.length; idx >= downTo; idx--) {
 
-        boolean doPrune = false;
-        boolean doCompile = false;
+      boolean doPrune = false;
+      boolean doCompile = false;
 
-        final UnCompiledNode<T> node = frontier[idx];
-        final UnCompiledNode<T> parent = frontier[idx-1];
+      final UnCompiledNode<T> node = frontier[idx];
+      final UnCompiledNode<T> parent = frontier[idx-1];
 
-        if (node.inputCount < minSuffixCount1) {
+      if (node.inputCount < minSuffixCount1) {
+        doPrune = true;
+        doCompile = true;
+      } else if (idx > prefixLenPlus1) {
+        // prune if parent's inputCount is less than suffixMinCount2
+        if (parent.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && parent.inputCount == 1 && idx > 1)) {
+          // my parent, about to be compiled, doesn't make the cut, so
+          // I'm definitely pruned 
+
+          // if minSuffixCount2 is 1, we keep only up
+          // until the 'distinguished edge', ie we keep only the
+          // 'divergent' part of the FST. if my parent, about to be
+          // compiled, has inputCount 1 then we are already past the
+          // distinguished edge.  NOTE: this only works if
+          // the FST outputs are not "compressible" (simple
+          // ords ARE compressible).
           doPrune = true;
-          doCompile = true;
-        } else if (idx > prefixLenPlus1) {
-          // prune if parent's inputCount is less than suffixMinCount2
-          if (parent.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && parent.inputCount == 1 && idx > 1)) {
-            // my parent, about to be compiled, doesn't make the cut, so
-            // I'm definitely pruned 
-
-            // if minSuffixCount2 is 1, we keep only up
-            // until the 'distinguished edge', ie we keep only the
-            // 'divergent' part of the FST. if my parent, about to be
-            // compiled, has inputCount 1 then we are already past the
-            // distinguished edge.  NOTE: this only works if
-            // the FST outputs are not "compressible" (simple
-            // ords ARE compressible).
-            doPrune = true;
-          } else {
-            // my parent, about to be compiled, does make the cut, so
-            // I'm definitely not pruned 
-            doPrune = false;
-          }
-          doCompile = true;
         } else {
-          // if pruning is disabled (count is 0) we can always
-          // compile current node
-          doCompile = minSuffixCount2 == 0;
+          // my parent, about to be compiled, does make the cut, so
+          // I'm definitely not pruned 
+          doPrune = false;
+        }
+        doCompile = true;
+      } else {
+        // if pruning is disabled (count is 0) we can always
+        // compile current node
+        doCompile = minSuffixCount2 == 0;
+      }
+
+      //System.out.println("    label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
+
+      if (node.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && node.inputCount == 1 && idx > 1)) {
+        // drop all arcs
+        for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
+          @SuppressWarnings({"rawtypes","unchecked"}) final UnCompiledNode<T> target =
+          (UnCompiledNode<T>) node.arcs[arcIdx].target;
+          target.clear();
         }
+        node.numArcs = 0;
+      }
 
-        //System.out.println("    label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune);
+      if (doPrune) {
+        // this node doesn't make it -- deref it
+        node.clear();
+        parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
+      } else {
 
-        if (node.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && node.inputCount == 1 && idx > 1)) {
-          // drop all arcs
-          for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
-            @SuppressWarnings({"rawtypes","unchecked"}) final UnCompiledNode<T> target =
-                (UnCompiledNode<T>) node.arcs[arcIdx].target;
-            target.clear();
-          }
-          node.numArcs = 0;
+        if (minSuffixCount2 != 0) {
+          compileAllTargets(node, lastInput.length-idx);
         }
+        final T nextFinalOutput = node.output;
 
-        if (doPrune) {
-          // this node doesn't make it -- deref it
-          node.clear();
-          parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
+        // We "fake" the node as being final if it has no
+        // outgoing arcs; in theory we could leave it
+        // as non-final (the FST can represent this), but
+        // FSTEnum, Util, etc., have trouble w/ non-final
+        // dead-end states:
+        final boolean isFinal = node.isFinal || node.numArcs == 0;
+
+        if (doCompile) {
+          // this node makes it and we now compile it.  first,
+          // compile any targets that were previously
+          // undecided:
+          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                             compileNode(node, 1+lastInput.length-idx),
+                             nextFinalOutput,
+                             isFinal);
         } else {
-
-          if (minSuffixCount2 != 0) {
-            compileAllTargets(node, lastInput.length-idx);
-          }
-          final T nextFinalOutput = node.output;
-
-          // We "fake" the node as being final if it has no
-          // outgoing arcs; in theory we could leave it
-          // as non-final (the FST can represent this), but
-          // FSTEnum, Util, etc., have trouble w/ non-final
-          // dead-end states:
-          final boolean isFinal = node.isFinal || node.numArcs == 0;
-
-          if (doCompile) {
-            // this node makes it and we now compile it.  first,
-            // compile any targets that were previously
-            // undecided:
-            parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
-                               compileNode(node, 1+lastInput.length-idx),
-                               nextFinalOutput,
-                               isFinal);
-          } else {
-            // replaceLast just to install
-            // nextFinalOutput/isFinal onto the arc
-            parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
-                               node,
-                               nextFinalOutput,
-                               isFinal);
-            // this node will stay in play for now, since we are
-            // undecided on whether to prune it.  later, it
-            // will be either compiled or pruned, so we must
-            // allocate a new node:
-            frontier[idx] = new UnCompiledNode<>(this, idx);
-          }
+          // replaceLast just to install
+          // nextFinalOutput/isFinal onto the arc
+          parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
+                             node,
+                             nextFinalOutput,
+                             isFinal);
+          // this node will stay in play for now, since we are
+          // undecided on whether to prune it.  later, it
+          // will be either compiled or pruned, so we must
+          // allocate a new node:
+          frontier[idx] = new UnCompiledNode<>(this, idx);
         }
       }
     }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java?rev=1613235&r1=1613234&r2=1613235&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/Test2BFST.java Thu Jul 24 18:23:06 2014
@@ -28,7 +28,6 @@ import org.apache.lucene.store.MMapDirec
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.TimeUnits;
 import org.apache.lucene.util.packed.PackedInts;
 import org.junit.Ignore;
@@ -56,7 +55,7 @@ public class Test2BFST extends LuceneTes
         Outputs<Object> outputs = NoOutputs.getSingleton();
         Object NO_OUTPUT = outputs.getNoOutput();
         final Builder<Object> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs,
-                                                      null, doPack, PackedInts.COMPACT, true, 15);
+                                                doPack, PackedInts.COMPACT, true, 15);
 
         int count = 0;
         Random r = new Random(seed);
@@ -138,7 +137,7 @@ public class Test2BFST extends LuceneTes
         System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=bytes");
         Outputs<BytesRef> outputs = ByteSequenceOutputs.getSingleton();
         final Builder<BytesRef> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs,
-                                                          null, doPack, PackedInts.COMPACT, true, 15);
+                                                  doPack, PackedInts.COMPACT, true, 15);
 
         byte[] outputBytes = new byte[20];
         BytesRef output = new BytesRef(outputBytes);
@@ -215,7 +214,7 @@ public class Test2BFST extends LuceneTes
         System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=long");
         Outputs<Long> outputs = PositiveIntOutputs.getSingleton();
         final Builder<Long> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs,
-                                                  null, doPack, PackedInts.COMPACT, true, 15);
+                                              doPack, PackedInts.COMPACT, true, 15);
 
         long output = 1;
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1613235&r1=1613234&r2=1613235&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Thu Jul 24 18:23:06 2014
@@ -326,7 +326,7 @@ public class TestFSTs extends LuceneTest
 
     final boolean doRewrite = random().nextBoolean();
 
-    Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, PackedInts.DEFAULT, true, 15);
+    Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, doRewrite, PackedInts.DEFAULT, true, 15);
 
     boolean storeOrd = random().nextBoolean();
     if (VERBOSE) {
@@ -469,7 +469,7 @@ public class TestFSTs extends LuceneTest
       this.outputs = outputs;
       this.doPack = doPack;
 
-      builder = new Builder<>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, PackedInts.DEFAULT, !noArcArrays, 15);
+      builder = new Builder<>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, doPack, PackedInts.DEFAULT, !noArcArrays, 15);
     }
 
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@@ -1113,7 +1113,7 @@ public class TestFSTs extends LuceneTest
   public void testFinalOutputOnEndState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
 
-    final Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), PackedInts.DEFAULT, true, 15);
+    final Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, random().nextBoolean(), PackedInts.DEFAULT, true, 15);
     builder.add(Util.toUTF32("stat", new IntsRef()), 17L);
     builder.add(Util.toUTF32("station", new IntsRef()), 10L);
     final FST<Long> fst = builder.finish();
@@ -1128,7 +1128,7 @@ public class TestFSTs extends LuceneTest
   public void testInternalFinalState() throws Exception {
     final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     final boolean willRewrite = random().nextBoolean();
-    final Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, PackedInts.DEFAULT, true, 15);
+    final Builder<Long> builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, willRewrite, PackedInts.DEFAULT, true, 15);
     builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRef()), outputs.getNoOutput());
     builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRef()), outputs.getNoOutput());
     final FST<Long> fst = builder.finish();