LUCENE-5841: simplify how block tree terms dict assigns terms to blocks

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1613161 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2014-07-24 15:46:31 +00:00
parent 6b5f5bab08
commit 8b026f6bd9
14 changed files with 929 additions and 1341 deletions

View File

@ -182,6 +182,9 @@ Optimizations
* LUCENE-5834: Empty sorted set and numeric doc values are now singletons. * LUCENE-5834: Empty sorted set and numeric doc values are now singletons.
(Adrien Grand) (Adrien Grand)
* LUCENE-5841: Improve performance of block tree terms dictionary when
assigning terms to blocks. (Mike McCandless)
Bug Fixes Bug Fixes
* LUCENE-5796: Fixes the Scorer.getChildren() method for two combinations * LUCENE-5796: Fixes the Scorer.getChildren() method for two combinations

View File

@ -132,7 +132,7 @@ public class TokenInfoDictionaryBuilder {
System.out.println(" encode..."); System.out.println(" encode...");
PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton(); PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton();
Builder<Long> fstBuilder = new Builder<>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, null, true, PackedInts.DEFAULT, true, 15); Builder<Long> fstBuilder = new Builder<>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, true, PackedInts.DEFAULT, true, 15);
IntsRef scratch = new IntsRef(); IntsRef scratch = new IntsRef();
long ord = -1; // first ord will be 0 long ord = -1; // first ord will be 0
String lastValue = null; String lastValue = null;

View File

@ -25,7 +25,7 @@ import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer; import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase; import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter; // javadocs
import org.apache.lucene.codecs.blocktreeords.FSTOrdsOutputs.Output; import org.apache.lucene.codecs.blocktreeords.FSTOrdsOutputs.Output;
import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfo;
@ -42,10 +42,10 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef; 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.Builder;
import org.apache.lucene.util.fst.BytesRefFSTEnum; import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST; import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.NoOutputs;
import org.apache.lucene.util.fst.Util; import org.apache.lucene.util.fst.Util;
import org.apache.lucene.util.packed.PackedInts; import org.apache.lucene.util.packed.PackedInts;
@ -89,8 +89,6 @@ import org.apache.lucene.util.packed.PackedInts;
public final class OrdsBlockTreeTermsWriter extends FieldsConsumer { public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
// private static boolean DEBUG = IDOrdsSegmentTermsEnum.DEBUG;
static final FSTOrdsOutputs FST_OUTPUTS = new FSTOrdsOutputs(); static final FSTOrdsOutputs FST_OUTPUTS = new FSTOrdsOutputs();
static final Output NO_OUTPUT = FST_OUTPUTS.getNoOutput(); static final Output NO_OUTPUT = FST_OUTPUTS.getNoOutput();
@ -105,7 +103,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
* #OrdsBlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */ * #OrdsBlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */
public final static int DEFAULT_MAX_BLOCK_SIZE = 48; 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; //private final static boolean SAVE_DOT_FILES = false;
static final int OUTPUT_FLAGS_NUM_BITS = 2; static final int OUTPUT_FLAGS_NUM_BITS = 2;
@ -273,22 +271,42 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
} }
private static final class PendingTerm extends PendingEntry { private static final class PendingTerm extends PendingEntry {
public final BytesRef term; public final byte[] termBytes;
// stats + metadata // stats + metadata
public final BlockTermState state; public final BlockTermState state;
public PendingTerm(BytesRef term, BlockTermState state) { public PendingTerm(BytesRef term, BlockTermState state) {
super(true); super(true);
this.term = term; this.termBytes = new byte[term.length];
System.arraycopy(term.bytes, term.offset, termBytes, 0, term.length);
this.state = state; this.state = state;
} }
@Override @Override
public String toString() { 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 SubIndex { private static final class SubIndex {
public final FST<Output> index; public final FST<Output> index;
public final long termOrdStart; public final long termOrdStart;
@ -309,7 +327,6 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
public final int floorLeadByte; public final int floorLeadByte;
public long totFloorTermCount; public long totFloorTermCount;
private final long totalTermCount; private final long totalTermCount;
private final IntsRef scratchIntsRef = new IntsRef();
public PendingBlock(BytesRef prefix, long fp, boolean hasTerms, long totalTermCount, public PendingBlock(BytesRef prefix, long fp, boolean hasTerms, long totalTermCount,
boolean isFloor, int floorLeadByte, List<SubIndex> subIndices) { boolean isFloor, int floorLeadByte, List<SubIndex> subIndices) {
@ -318,6 +335,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
this.fp = fp; this.fp = fp;
this.hasTerms = hasTerms; this.hasTerms = hasTerms;
this.totalTermCount = totalTermCount; this.totalTermCount = totalTermCount;
assert totalTermCount > 0;
this.isFloor = isFloor; this.isFloor = isFloor;
this.floorLeadByte = floorLeadByte; this.floorLeadByte = floorLeadByte;
this.subIndices = subIndices; this.subIndices = subIndices;
@ -325,25 +343,26 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
@Override @Override
public String toString() { 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; assert scratchBytes.getFilePointer() == 0;
// TODO: try writing the leading vLong in MSB order // TODO: try writing the leading vLong in MSB order
// (opposite of what Lucene does today), for better // (opposite of what Lucene does today), for better
// outputs sharing in the FST // outputs sharing in the FST
//System.out.println("\ncompileIndex isFloor=" + isFloor + " numTerms=" + totalTermCount);
long lastSumTotalTermCount = 0; long lastSumTotalTermCount = 0;
long sumTotalTermCount = totalTermCount; long sumTotalTermCount = totalTermCount;
scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor)); scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor));
if (isFloor) { if (isFloor) {
scratchBytes.writeVInt(floorBlocks.size()); scratchBytes.writeVInt(blocks.size()-1);
for (PendingBlock sub : floorBlocks) { for (int i=1;i<blocks.size();i++) {
PendingBlock sub = blocks.get(i);
assert sub.floorLeadByte != -1; assert sub.floorLeadByte != -1;
//if (DEBUG) { //if (DEBUG) {
// System.out.println(" write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff)); // System.out.println(" write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff));
@ -360,7 +379,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
final Builder<Output> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1, final Builder<Output> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1,
0, 0, true, false, Integer.MAX_VALUE, 0, 0, true, false, Integer.MAX_VALUE,
FST_OUTPUTS, null, false, FST_OUTPUTS, false,
PackedInts.COMPACT, true, 15); PackedInts.COMPACT, true, 15);
//if (DEBUG) { //if (DEBUG) {
// System.out.println(" compile index for prefix=" + prefix); // System.out.println(" compile index for prefix=" + prefix);
@ -377,31 +396,22 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
// Copy over index for all sub-blocks // Copy over index for all sub-blocks
if (subIndices != null) { long termOrdOffset = 0;
for(SubIndex subIndex : subIndices) { for(PendingBlock block : blocks) {
//System.out.println(" append subIndex: termOrdStart=" + subIndex.termOrdStart); if (block.subIndices != null) {
append(indexBuilder, subIndex.index, subIndex.termOrdStart); for(SubIndex subIndex : block.subIndices) {
} append(indexBuilder, subIndex.index, termOrdOffset + subIndex.termOrdStart, scratchIntsRef);
}
if (floorBlocks != null) {
long termOrdOffset = totalTermCount;
for (PendingBlock sub : floorBlocks) {
if (sub.subIndices != null) {
for(SubIndex subIndex : sub.subIndices) {
append(indexBuilder, subIndex.index, termOrdOffset + subIndex.termOrdStart);
}
} }
sub.subIndices = null; block.subIndices = null;
termOrdOffset += sub.totalTermCount;
} }
totFloorTermCount = termOrdOffset; termOrdOffset += block.totalTermCount;
} else {
totFloorTermCount = sumTotalTermCount;
} }
totFloorTermCount = termOrdOffset;
assert sumTotalTermCount == totFloorTermCount;
index = indexBuilder.finish(); index = indexBuilder.finish();
subIndices = null; assert subIndices == null;
/* /*
Writer w = new OutputStreamWriter(new FileOutputStream("out.dot")); Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
@ -414,7 +424,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
// TODO: maybe we could add bulk-add method to // TODO: maybe we could add bulk-add method to
// Builder? Takes FST and unions it w/ current // Builder? Takes FST and unions it w/ current
// FST. // FST.
private void append(Builder<Output> builder, FST<Output> subIndex, long termOrdOffset) throws IOException { private void append(Builder<Output> builder, FST<Output> subIndex, long termOrdOffset, IntsRef scratchIntsRef) throws IOException {
final BytesRefFSTEnum<Output> subIndexEnum = new BytesRefFSTEnum<>(subIndex); final BytesRefFSTEnum<Output> subIndexEnum = new BytesRefFSTEnum<>(subIndex);
BytesRefFSTEnum.InputOutput<Output> indexEnt; BytesRefFSTEnum.InputOutput<Output> indexEnt;
while ((indexEnt = subIndexEnum.next()) != null) { while ((indexEnt = subIndexEnum.next()) != null) {
@ -430,7 +440,8 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
} }
} }
final RAMOutputStream scratchBytes = new RAMOutputStream(); private final RAMOutputStream scratchBytes = new RAMOutputStream();
private final IntsRef scratchIntsRef = new IntsRef();
class TermsWriter { class TermsWriter {
private final FieldInfo fieldInfo; private final FieldInfo fieldInfo;
@ -441,395 +452,199 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
long sumDocFreq; long sumDocFreq;
long indexStartFP; long indexStartFP;
// Used only to partition terms into the block tree; we // Records index into pending where the current prefix at that
// don't pull an FST from this builder: // length "started"; for example, if current term starts with 't',
private final NoOutputs noOutputs; // startsByPrefix[0] is the index into pending for the first
private final Builder<Object> blockBuilder; // 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];
// PendingTerm or PendingBlock: 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<>(); private final List<PendingEntry> pending = new ArrayList<>();
// Index into pending of most recently written block // Reused in writeBlocks:
private int lastBlockIndex = -1; private final List<PendingBlock> newBlocks = new ArrayList<>();
// Re-used when segmenting a too-large block into floor /** Writes the top count entries in pending, using prevTerm to compute the prefix. */
// blocks: void writeBlocks(int prefixLength, int count) throws IOException {
private int[] subBytes = new int[10];
private int[] subTermCounts = new int[10];
private int[] subTermCountSums = new int[10];
private int[] subSubCounts = new int[10];
// This class assigns terms to blocks "naturally", ie, assert count > 0;
// according to the number of terms under a given prefix
// that we encounter:
private class FindBlocks extends Builder.FreezeTail<Object> {
@Override /*
public void freeze(final Builder.UnCompiledNode<Object>[] frontier, int prefixLenPlus1, final IntsRef lastInput) throws IOException { if (DEBUG) {
BytesRef br = new BytesRef(lastTerm.bytes);
//if (DEBUG) System.out.println(" freeze prefixLenPlus1=" + prefixLenPlus1); br.offset = lastTerm.offset;
br.length = prefixLength;
for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) { System.out.println("writeBlocks: " + br.utf8ToString() + " count=" + count);
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);
}
} }
} */
// Write the top count entries on the pending stack as // Root block better write all remaining pending entries:
// one or more blocks. Returns how many blocks were assert prefixLength > 0 || count == pending.size();
// written. 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 { int lastSuffixLeadLabel = -1;
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);
} 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 // True if we saw at least one term in this block (we record if a block
// in each block, to make floor blocks authoritative // 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;
/* int start = pending.size()-count;
if (DEBUG) { int end = pending.size();
final BytesRef prefix = new BytesRef(prefixLength); int nextBlockStart = start;
for(int m=0;m<prefixLength;m++) { int nextFloorLeadLabel = -1;
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=" + toString(prefix) + " " + prefix + " count=" + count + " pending.size()=" + pending.size());
}
*/
//System.out.println("\nwbs count=" + count);
final int savLabel = prevTerm.ints[prevTerm.offset + prefixLength]; for (int i=start; i<end; i++) {
// Count up how many items fall under PendingEntry ent = pending.get(i);
// 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;
for(PendingEntry ent : slice) { int suffixLeadLabel;
// First byte in the suffix of this term if (ent.isTerm) {
final int suffixLeadLabel; PendingTerm term = (PendingTerm) ent;
if (ent.isTerm) { if (term.termBytes.length == prefixLength) {
PendingTerm term = (PendingTerm) ent; // Suffix is 0, i.e. prefix 'foo' and term is
if (term.term.length == prefixLength) { // 'foo' so the term has empty string suffix
// Suffix is 0, ie prefix 'foo' and term is // in this block
// 'foo' so the term has empty string suffix assert lastSuffixLeadLabel == -1;
// in this block suffixLeadLabel = -1;
assert lastSuffixLeadLabel == -1;
assert numSubs == 0;
suffixLeadLabel = -1;
} else {
suffixLeadLabel = term.term.bytes[term.term.offset + prefixLength] & 0xff;
}
} else { } else {
PendingBlock block = (PendingBlock) ent; suffixLeadLabel = term.termBytes[prefixLength] & 0xff;
assert block.prefix.length > prefixLength;
suffixLeadLabel = block.prefix.bytes[block.prefix.offset + prefixLength] & 0xff;
} }
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++;
}
if (ent.isTerm) {
termCount++;
} else {
subCount++;
}
}
if (subBytes.length == numSubs) {
subBytes = ArrayUtil.grow(subBytes);
subTermCounts = ArrayUtil.grow(subTermCounts);
subSubCounts = ArrayUtil.grow(subSubCounts);
}
subBytes[numSubs] = lastSuffixLeadLabel;
subTermCounts[numSubs] = termCount;
subSubCounts[numSubs] = subCount;
numSubs++;
/*
if (lastSuffixLeadLabel == -1) {
System.out.println(" sub " + -1 + " termCount=" + termCount + " subCount=" + subCount);
} else { } else {
System.out.println(" sub " + Integer.toHexString(lastSuffixLeadLabel) + " termCount=" + termCount + " subCount=" + subCount); 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 (subTermCountSums.length < numSubs) { if (suffixLeadLabel != lastSuffixLeadLabel) {
subTermCountSums = ArrayUtil.grow(subTermCountSums, numSubs); 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));
// Roll up (backwards) the termCounts; postings impl hasTerms = false;
// needs this to know where to pull the term slice hasSubBlocks = false;
// from its pending terms stack: nextFloorLeadLabel = suffixLeadLabel;
int sum = 0; nextBlockStart = i;
for(int idx=numSubs-1;idx>=0;idx--) {
sum += subTermCounts[idx];
subTermCountSums[idx] = sum;
}
// 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(" = " + 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];
if (curStart == 0) {
break;
}
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;
}
} }
lastSuffixLeadLabel = suffixLeadLabel;
} }
prevTerm.ints[prevTerm.offset + prefixLength] = savLabel; if (ent.isTerm) {
hasTerms = true;
assert firstBlock != null; } else {
firstBlock.compileIndex(floorBlocks, scratchBytes); hasSubBlocks = true;
}
pending.add(firstBlock);
//if (DEBUG) System.out.println(" done pending.size()=" + pending.size());
} }
lastBlockIndex = pending.size()-1;
// 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));
}
assert newBlocks.isEmpty() == false;
PendingBlock firstBlock = newBlocks.get(0);
assert firstBlock.isFloor || newBlocks.size() == 1;
firstBlock.compileIndex(newBlocks, scratchBytes, scratchIntsRef);
// Remove slice from the top of the pending stack, that we just wrote:
pending.subList(pending.size()-count, pending.size()).clear();
// Append new block
pending.add(firstBlock);
newBlocks.clear();
} }
// BytesRef prefix; /** 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 {
// for debugging assert end > start;
@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();
}
}
// Writes all entries in the pending slice as a single long startFP = out.getFilePointer();
// block:
private PendingBlock writeBlock(IntsRef prevTerm, int prefixLength, int indexPrefixLength, int startBackwards, int length,
int futureTermCount, boolean isFloor, int floorLeadByte, boolean isLastInFloor) throws IOException {
assert length > 0; // if (DEBUG) System.out.println(" writeBlock fp=" + startFP + " isFloor=" + isFloor + " floorLeadLabel=" + floorLeadLabel + " start=" + start + " end=" + end + " hasTerms=" + hasTerms + " hasSubBlocks=" + hasSubBlocks);
final int start = pending.size()-startBackwards; boolean hasFloorLeadLabel = isFloor && floorLeadLabel != -1;
assert start >= 0: "pending.size()=" + pending.size() + " startBackwards=" + startBackwards + " length=" + length; final BytesRef prefix = new BytesRef(prefixLength + (hasFloorLeadLabel ? 1 : 0));
System.arraycopy(lastTerm.bytes, 0, prefix.bytes, 0, prefixLength);
final List<PendingEntry> slice = pending.subList(start, start + length); prefix.length = prefixLength;
final long startFP = out.getFilePointer();
// System.out.println("\nwriteBlock field=" + fieldInfo.name + " seg=" + segment + " prefixLength=" + prefixLength + " floorLeadByte=" + floorLeadByte + " isLastInFloor=" + isLastInFloor + " length=" + length + " startFP=" + startFP);
final BytesRef prefix = new BytesRef(indexPrefixLength);
for(int m=0;m<indexPrefixLength;m++) {
prefix.bytes[m] = (byte) prevTerm.ints[m];
}
prefix.length = indexPrefixLength;
// System.out.println(" prefix=" + toString(prefix));
// this.prefix = prefix;
// Write block header: // 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) { // 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;
}
// System.out.println(" isLeaf=" + isLeafBlock);
final List<SubIndex> subIndices; final List<SubIndex> subIndices;
// We optimize the leaf block case (block has only terms), writing a more
// compact format in this case:
boolean isLeafBlock = hasSubBlocks == false;
// Number of terms in this block // Number of terms in this block
int termCount; int termCount;
// Number of terms in this block and all sub-blocks (recursively) // Number of terms in this block and all sub-blocks (recursively)
long totalTermCount; long totalTermCount;
long[] longs = new long[longsSize];
boolean absolute = true; boolean absolute = true;
int countx = 0;
if (isLeafBlock) { if (isLeafBlock) {
// Only terms:
subIndices = null; subIndices = null;
for (PendingEntry ent : slice) { for (int i=start;i<end;i++) {
assert ent.isTerm; PendingEntry ent = pending.get(i);
assert ent.isTerm: "i=" + i;
PendingTerm term = (PendingTerm) ent; PendingTerm term = (PendingTerm) ent;
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
BlockTermState state = term.state; BlockTermState state = term.state;
final int suffix = term.term.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
/* /*
if (DEBUG) { if (DEBUG) {
BytesRef suffixBytes = new BytesRef(suffix); BytesRef suffixBytes = new BytesRef(suffix);
System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix); System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
suffixBytes.length = suffix; suffixBytes.length = suffix;
System.out.println(" " + (countx++) + ": write term suffix=" + toString(suffixBytes)); System.out.println(" write term suffix=" + suffixBytes);
} }
*/ */
// For leaf block we write suffix straight // For leaf block we write suffix straight
suffixWriter.writeVInt(suffix); 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: // Write term stats, to separate byte[] blob:
statsWriter.writeVInt(state.docFreq); statsWriter.writeVInt(state.docFreq);
@ -837,7 +652,6 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
assert state.totalTermFreq >= state.docFreq: state.totalTermFreq + " vs " + state.docFreq; assert state.totalTermFreq >= state.docFreq: state.totalTermFreq + " vs " + state.docFreq;
statsWriter.writeVLong(state.totalTermFreq - state.docFreq); statsWriter.writeVLong(state.totalTermFreq - state.docFreq);
} }
// System.out.println(" dF=" + state.docFreq + " tTF=" + state.totalTermFreq);
// Write term meta data // Write term meta data
postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute); postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
@ -849,29 +663,33 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
bytesWriter.reset(); bytesWriter.reset();
absolute = false; absolute = false;
} }
termCount = length; termCount = end-start;
totalTermCount = length; totalTermCount = end-start;
} else { } else {
// Mixed terms and sub-blocks:
subIndices = new ArrayList<>(); subIndices = new ArrayList<>();
termCount = 0; termCount = 0;
totalTermCount = 0; totalTermCount = 0;
for (PendingEntry ent : slice) { for (int i=start;i<end;i++) {
PendingEntry ent = pending.get(i);
if (ent.isTerm) { if (ent.isTerm) {
PendingTerm term = (PendingTerm) ent; PendingTerm term = (PendingTerm) ent;
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
BlockTermState state = term.state; BlockTermState state = term.state;
final int suffix = term.term.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
/* /*
if (DEBUG) { if (DEBUG) {
BytesRef suffixBytes = new BytesRef(suffix); BytesRef suffixBytes = new BytesRef(suffix);
System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix); System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
suffixBytes.length = suffix; suffixBytes.length = suffix;
System.out.println(" " + (countx++) + ": write term suffix=" + toString(suffixBytes) + " termOrd=" + totalTermCount); System.out.println(" write term suffix=" + suffixBytes);
} }
*/ */
// For non-leaf block we borrow 1 bit to record // For non-leaf block we borrow 1 bit to record
// if entry is term or sub-block // if entry is term or sub-block
suffixWriter.writeVInt(suffix<<1); 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: // Write term stats, to separate byte[] blob:
statsWriter.writeVInt(state.docFreq); statsWriter.writeVInt(state.docFreq);
@ -902,27 +720,30 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
totalTermCount++; totalTermCount++;
} else { } else {
PendingBlock block = (PendingBlock) ent; PendingBlock block = (PendingBlock) ent;
assert StringHelper.startsWith(block.prefix, prefix);
final int suffix = block.prefix.length - prefixLength; final int suffix = block.prefix.length - prefixLength;
assert suffix > 0; assert suffix > 0;
// For non-leaf block we steal 1 bit to record // For non-leaf block we borrow 1 bit to record
// if entry is term or sub-block // if entry is term or sub-block
suffixWriter.writeVInt((suffix<<1)|1); suffixWriter.writeVInt((suffix<<1)|1);
suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix); suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix);
assert block.fp < startFP;
suffixWriter.writeVLong(startFP - block.fp); assert floorLeadLabel == -1 || (block.prefix.bytes[prefixLength] & 0xff) >= floorLeadLabel;
assert block.fp < startFP;
/* /*
if (DEBUG) { if (DEBUG) {
BytesRef suffixBytes = new BytesRef(suffix); BytesRef suffixBytes = new BytesRef(suffix);
System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix); System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
suffixBytes.length = suffix; suffixBytes.length = suffix;
System.out.println(" " + (countx++) + ": write sub-block suffix=" + toString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor + " totFloorTermCount=" + block.totFloorTermCount); System.out.println(" write sub-block suffix=" + brToString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor);
} }
*/ */
suffixWriter.writeVLong(startFP - block.fp);
suffixWriter.writeVLong(block.totFloorTermCount); suffixWriter.writeVLong(block.totFloorTermCount);
subIndices.add(new SubIndex(block.index, totalTermCount)); subIndices.add(new SubIndex(block.index, totalTermCount));
totalTermCount += block.totFloorTermCount; totalTermCount += block.totFloorTermCount;
@ -943,7 +764,6 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
// Write term stats byte[] blob // Write term stats byte[] blob
out.writeVInt((int) statsWriter.getFilePointer()); out.writeVInt((int) statsWriter.getFilePointer());
//System.out.println("write stats @ fp=" + out.getFilePointer());
statsWriter.writeTo(out); statsWriter.writeTo(out);
statsWriter.reset(); statsWriter.reset();
@ -952,55 +772,42 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
metaWriter.writeTo(out); metaWriter.writeTo(out);
metaWriter.reset(); metaWriter.reset();
// Remove slice replaced by block:
slice.clear();
if (lastBlockIndex >= start) {
if (lastBlockIndex < start+length) {
lastBlockIndex = start;
} else {
lastBlockIndex -= length;
}
}
// if (DEBUG) { // if (DEBUG) {
// System.out.println(" fpEnd=" + out.getFilePointer()); // System.out.println(" fpEnd=" + out.getFilePointer());
// } // }
return new PendingBlock(prefix, startFP, termCount != 0, totalTermCount, isFloor, floorLeadByte, subIndices); if (hasFloorLeadLabel) {
// We already allocated to length+1 above:
prefix.bytes[prefix.length++] = (byte) floorLeadLabel;
}
return new PendingBlock(prefix, startFP, hasTerms, totalTermCount, isFloor, floorLeadLabel, subIndices);
} }
TermsWriter(FieldInfo fieldInfo) { TermsWriter(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo; this.fieldInfo = fieldInfo;
docsSeen = new FixedBitSet(maxDoc); 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.longsSize = postingsWriter.setField(fieldInfo);
this.longs = new long[longsSize];
} }
private final IntsRef scratchIntsRef = new IntsRef();
/** Writes one term's worth of postings. */ /** Writes one term's worth of postings. */
public void write(BytesRef text, TermsEnum termsEnum) throws IOException { public void write(BytesRef text, TermsEnum termsEnum) 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());
}
*/
BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen); BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen);
if (state != null) { if (state != null) {
assert state.docFreq != 0; assert state.docFreq != 0;
assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter; assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter;
sumDocFreq += state.docFreq; sumDocFreq += state.docFreq;
sumTotalTermFreq += state.totalTermFreq; sumTotalTermFreq += state.totalTermFreq;
blockBuilder.add(Util.toIntsRef(text, scratchIntsRef), noOutputs.getNoOutput()); pushTerm(text);
PendingTerm term = new PendingTerm(BytesRef.deepCopyOf(text), state); PendingTerm term = new PendingTerm(BytesRef.deepCopyOf(text), state);
pending.add(term); pending.add(term);
@ -1008,10 +815,52 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
} }
} }
/** 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);
}
// Finishes all terms in this field // Finishes all terms in this field
public void finish(BytesRef minTerm, BytesRef maxTerm) throws IOException { public void finish(BytesRef minTerm, BytesRef maxTerm) throws IOException {
if (numTerms > 0) { if (numTerms > 0) {
blockBuilder.finish(); // if (DEBUG) System.out.println("BTTW.finish pending.size()=" + pending.size());
// 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: // We better have one final "root" block:
assert pending.size() == 1 && !pending.get(0).isTerm: "pending.size()=" + pending.size() + " pending=" + pending; assert pending.size() == 1 && !pending.get(0).isTerm: "pending.size()=" + pending.size() + " pending=" + pending;

View File

@ -121,7 +121,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
this.field = field; this.field = field;
this.doPackFST = doPackFST; this.doPackFST = doPackFST;
this.acceptableOverheadRatio = acceptableOverheadRatio; this.acceptableOverheadRatio = acceptableOverheadRatio;
builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doPackFST, acceptableOverheadRatio, true, 15); builder = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, doPackFST, acceptableOverheadRatio, true, 15);
} }
private class PostingsWriter { private class PostingsWriter {

View File

@ -109,6 +109,9 @@ public class TestOrdsBlockTree extends BasePostingsFormatTestCase {
doc.add(newTextField("field", term, Field.Store.NO)); doc.add(newTextField("field", term, Field.Store.NO));
w.addDocument(doc); w.addDocument(doc);
} }
if (VERBOSE) {
System.out.println("TEST: now forceMerge");
}
w.forceMerge(1); w.forceMerge(1);
IndexReader r = w.getReader(); IndexReader r = w.getReader();
TermsEnum te = MultiFields.getTerms(r, "field").iterator(null); TermsEnum te = MultiFields.getTerms(r, "field").iterator(null);

View File

@ -41,11 +41,11 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef; 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.Builder;
import org.apache.lucene.util.fst.ByteSequenceOutputs; import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.BytesRefFSTEnum; import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST; 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.Outputs;
import org.apache.lucene.util.fst.Util; import org.apache.lucene.util.fst.Util;
import org.apache.lucene.util.packed.PackedInts; import org.apache.lucene.util.packed.PackedInts;
@ -204,7 +204,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
* #BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */ * #BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */
public final static int DEFAULT_MAX_BLOCK_SIZE = 48; 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; //private final static boolean SAVE_DOT_FILES = false;
static final int OUTPUT_FLAGS_NUM_BITS = 2; static final int OUTPUT_FLAGS_NUM_BITS = 2;
@ -277,6 +277,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
} }
private final List<FieldMetaData> fields = new ArrayList<>(); private final List<FieldMetaData> fields = new ArrayList<>();
// private final String segment; // private final String segment;
/** Create a new writer. The number of items (terms or /** Create a new writer. The number of items (terms or
@ -322,7 +323,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
writeIndexHeader(indexOut); writeIndexHeader(indexOut);
this.postingsWriter = postingsWriter; this.postingsWriter = postingsWriter;
// segment = state.segmentName; // segment = state.segmentInfo.name;
// System.out.println("BTW.init seg=" + state.segmentName); // System.out.println("BTW.init seg=" + state.segmentName);
@ -404,22 +405,42 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
} }
private static final class PendingTerm extends PendingEntry { private static final class PendingTerm extends PendingEntry {
public final BytesRef term; public final byte[] termBytes;
// stats + metadata // stats + metadata
public final BlockTermState state; public final BlockTermState state;
public PendingTerm(BytesRef term, BlockTermState state) { public PendingTerm(BytesRef term, BlockTermState state) {
super(true); super(true);
this.term = term; this.termBytes = new byte[term.length];
System.arraycopy(term.bytes, term.offset, termBytes, 0, term.length);
this.state = state; this.state = state;
} }
@Override @Override
public String toString() { 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 { private static final class PendingBlock extends PendingEntry {
public final BytesRef prefix; public final BytesRef prefix;
public final long fp; public final long fp;
@ -428,7 +449,6 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
public final boolean hasTerms; public final boolean hasTerms;
public final boolean isFloor; public final boolean isFloor;
public final int floorLeadByte; 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) { public PendingBlock(BytesRef prefix, long fp, boolean hasTerms, boolean isFloor, int floorLeadByte, List<FST<BytesRef>> subIndices) {
super(false); super(false);
@ -442,12 +462,13 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
@Override @Override
public String toString() { 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; assert scratchBytes.getFilePointer() == 0;
@ -456,8 +477,9 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
// outputs sharing in the FST // outputs sharing in the FST
scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor)); scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor));
if (isFloor) { if (isFloor) {
scratchBytes.writeVInt(floorBlocks.size()); scratchBytes.writeVInt(blocks.size()-1);
for (PendingBlock sub : floorBlocks) { for (int i=1;i<blocks.size();i++) {
PendingBlock sub = blocks.get(i);
assert sub.floorLeadByte != -1; assert sub.floorLeadByte != -1;
//if (DEBUG) { //if (DEBUG) {
// System.out.println(" write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff)); // System.out.println(" write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff));
@ -470,9 +492,9 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton(); final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
final Builder<BytesRef> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1, final Builder<BytesRef> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1,
0, 0, true, false, Integer.MAX_VALUE, 0, 0, true, false, Integer.MAX_VALUE,
outputs, null, false, outputs, false,
PackedInts.COMPACT, true, 15); PackedInts.COMPACT, true, 15);
//if (DEBUG) { //if (DEBUG) {
// System.out.println(" compile index for prefix=" + prefix); // System.out.println(" compile index for prefix=" + prefix);
//} //}
@ -484,26 +506,18 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
scratchBytes.reset(); scratchBytes.reset();
// Copy over index for all sub-blocks // Copy over index for all sub-blocks
for(PendingBlock block : blocks) {
if (subIndices != null) { if (block.subIndices != null) {
for(FST<BytesRef> subIndex : subIndices) { for(FST<BytesRef> subIndex : block.subIndices) {
append(indexBuilder, subIndex); append(indexBuilder, subIndex, scratchIntsRef);
}
}
if (floorBlocks != null) {
for (PendingBlock sub : floorBlocks) {
if (sub.subIndices != null) {
for(FST<BytesRef> subIndex : sub.subIndices) {
append(indexBuilder, subIndex);
}
} }
sub.subIndices = null; block.subIndices = null;
} }
} }
index = indexBuilder.finish(); index = indexBuilder.finish();
subIndices = null;
assert subIndices == null;
/* /*
Writer w = new OutputStreamWriter(new FileOutputStream("out.dot")); Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
@ -516,7 +530,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
// TODO: maybe we could add bulk-add method to // TODO: maybe we could add bulk-add method to
// Builder? Takes FST and unions it w/ current // Builder? Takes FST and unions it w/ current
// FST. // 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); final BytesRefFSTEnum<BytesRef> subIndexEnum = new BytesRefFSTEnum<>(subIndex);
BytesRefFSTEnum.InputOutput<BytesRef> indexEnt; BytesRefFSTEnum.InputOutput<BytesRef> indexEnt;
while((indexEnt = subIndexEnum.next()) != null) { while((indexEnt = subIndexEnum.next()) != null) {
@ -528,7 +542,8 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
} }
} }
final RAMOutputStream scratchBytes = new RAMOutputStream(); private final RAMOutputStream scratchBytes = new RAMOutputStream();
private final IntsRef scratchIntsRef = new IntsRef();
class TermsWriter { class TermsWriter {
private final FieldInfo fieldInfo; private final FieldInfo fieldInfo;
@ -539,381 +554,196 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
long sumDocFreq; long sumDocFreq;
long indexStartFP; long indexStartFP;
// Used only to partition terms into the block tree; we // Records index into pending where the current prefix at that
// don't pull an FST from this builder: // length "started"; for example, if current term starts with 't',
private final NoOutputs noOutputs; // startsByPrefix[0] is the index into pending for the first
private final Builder<Object> blockBuilder; // 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];
// PendingTerm or PendingBlock: 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<>(); private final List<PendingEntry> pending = new ArrayList<>();
// Index into pending of most recently written block // Reused in writeBlocks:
private int lastBlockIndex = -1; private final List<PendingBlock> newBlocks = new ArrayList<>();
// Re-used when segmenting a too-large block into floor /** Writes the top count entries in pending, using prevTerm to compute the prefix. */
// blocks: void writeBlocks(int prefixLength, int count) throws IOException {
private int[] subBytes = new int[10];
private int[] subTermCounts = new int[10];
private int[] subTermCountSums = new int[10];
private int[] subSubCounts = new int[10];
// This class assigns terms to blocks "naturally", ie, assert count > 0;
// according to the number of terms under a given prefix
// that we encounter:
private class FindBlocks extends Builder.FreezeTail<Object> {
@Override /*
public void freeze(final Builder.UnCompiledNode<Object>[] frontier, int prefixLenPlus1, final IntsRef lastInput) throws IOException { if (DEBUG) {
BytesRef br = new BytesRef(lastTerm.bytes);
//if (DEBUG) System.out.println(" freeze prefixLenPlus1=" + prefixLenPlus1); br.offset = lastTerm.offset;
br.length = prefixLength;
for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) { System.out.println("writeBlocks: " + br.utf8ToString() + " count=" + count);
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);
}
} }
} */
// Write the top count entries on the pending stack as // Root block better write all remaining pending entries:
// one or more blocks. If the entry count is <= maxItemsPerBlock assert prefixLength > 0 || count == pending.size();
// 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 { int lastSuffixLeadLabel = -1;
// 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 // True if we saw at least one term in this block (we record if a block
// in each block, to make floor blocks authoritative // 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;
//if (DEBUG) { int start = pending.size()-count;
// final BytesRef prefix = new BytesRef(prefixLength); int end = pending.size();
// for(int m=0;m<prefixLength;m++) { int nextBlockStart = start;
// prefix.bytes[m] = (byte) prevTerm.ints[m]; int nextFloorLeadLabel = -1;
// }
// 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);
final int savLabel = prevTerm.ints[prevTerm.offset + prefixLength]; for (int i=start; i<end; i++) {
// Count up how many items fall under PendingEntry ent = pending.get(i);
// 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;
for(PendingEntry ent : slice) { int suffixLeadLabel;
// First byte in the suffix of this term if (ent.isTerm) {
final int suffixLeadLabel; PendingTerm term = (PendingTerm) ent;
if (ent.isTerm) { if (term.termBytes.length == prefixLength) {
PendingTerm term = (PendingTerm) ent; // Suffix is 0, i.e. prefix 'foo' and term is
if (term.term.length == prefixLength) { // 'foo' so the term has empty string suffix
// Suffix is 0, ie prefix 'foo' and term is // in this block
// 'foo' so the term has empty string suffix assert lastSuffixLeadLabel == -1;
// in this block suffixLeadLabel = -1;
assert lastSuffixLeadLabel == -1;
assert numSubs == 0;
suffixLeadLabel = -1;
} else {
suffixLeadLabel = term.term.bytes[term.term.offset + prefixLength] & 0xff;
}
} else { } else {
PendingBlock block = (PendingBlock) ent; suffixLeadLabel = term.termBytes[prefixLength] & 0xff;
assert block.prefix.length > prefixLength;
suffixLeadLabel = block.prefix.bytes[block.prefix.offset + prefixLength] & 0xff;
} }
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++;
}
if (ent.isTerm) {
termCount++;
} else {
subCount++;
}
}
if (subBytes.length == numSubs) {
subBytes = ArrayUtil.grow(subBytes);
subTermCounts = ArrayUtil.grow(subTermCounts);
subSubCounts = ArrayUtil.grow(subSubCounts);
}
subBytes[numSubs] = lastSuffixLeadLabel;
subTermCounts[numSubs] = termCount;
subSubCounts[numSubs] = subCount;
numSubs++;
/*
if (lastSuffixLeadLabel == -1) {
System.out.println(" sub " + -1 + " termCount=" + termCount + " subCount=" + subCount);
} else { } else {
System.out.println(" sub " + Integer.toHexString(lastSuffixLeadLabel) + " termCount=" + termCount + " subCount=" + subCount); 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 (subTermCountSums.length < numSubs) { if (suffixLeadLabel != lastSuffixLeadLabel) {
subTermCountSums = ArrayUtil.grow(subTermCountSums, numSubs); 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));
// Roll up (backwards) the termCounts; postings impl hasTerms = false;
// needs this to know where to pull the term slice hasSubBlocks = false;
// from its pending terms stack: nextFloorLeadLabel = suffixLeadLabel;
int sum = 0; nextBlockStart = i;
for(int idx=numSubs-1;idx>=0;idx--) {
sum += subTermCounts[idx];
subTermCountSums[idx] = sum;
}
// 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];
if (curStart == 0) {
break;
}
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;
}
} }
lastSuffixLeadLabel = suffixLeadLabel;
} }
prevTerm.ints[prevTerm.offset + prefixLength] = savLabel; if (ent.isTerm) {
hasTerms = true;
assert firstBlock != null; } else {
firstBlock.compileIndex(floorBlocks, scratchBytes); hasSubBlocks = true;
}
pending.add(firstBlock);
//if (DEBUG) System.out.println(" done pending.size()=" + pending.size());
} }
lastBlockIndex = pending.size()-1;
// 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));
}
assert newBlocks.isEmpty() == false;
PendingBlock firstBlock = newBlocks.get(0);
assert firstBlock.isFloor || newBlocks.size() == 1;
firstBlock.compileIndex(newBlocks, scratchBytes, scratchIntsRef);
// Remove slice from the top of the pending stack, that we just wrote:
pending.subList(pending.size()-count, pending.size()).clear();
// Append new block
pending.add(firstBlock);
newBlocks.clear();
} }
// for debugging /** Writes the specified slice (start is inclusive, end is exclusive)
@SuppressWarnings("unused") * from pending stack as a new block. If isFloor is true, there
private String toString(BytesRef b) { * were too many (more than maxItemsInBlock) entries sharing the
try { * same prefix, and so we broke it into multiple floor blocks where
return b.utf8ToString() + " " + b; * we record the starting label of the suffix of each floor block. */
} catch (Throwable t) { private PendingBlock writeBlock(int prefixLength, boolean isFloor, int floorLeadLabel, int start, int end, boolean hasTerms, boolean hasSubBlocks) throws IOException {
// 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();
}
}
// Writes all entries in the pending slice as a single assert end > start;
// block:
private PendingBlock writeBlock(IntsRef prevTerm, int prefixLength, int indexPrefixLength, int startBackwards, int length,
int futureTermCount, boolean isFloor, int floorLeadByte, boolean isLastInFloor) throws IOException {
assert length > 0; long startFP = out.getFilePointer();
final int start = pending.size()-startBackwards; // if (DEBUG) System.out.println(" writeBlock fp=" + startFP + " isFloor=" + isFloor + " floorLeadLabel=" + floorLeadLabel + " start=" + start + " end=" + end + " hasTerms=" + hasTerms + " hasSubBlocks=" + hasSubBlocks);
assert start >= 0: "pending.size()=" + pending.size() + " startBackwards=" + startBackwards + " length=" + length; boolean hasFloorLeadLabel = isFloor && floorLeadLabel != -1;
final List<PendingEntry> slice = pending.subList(start, start + length); final BytesRef prefix = new BytesRef(prefixLength + (hasFloorLeadLabel ? 1 : 0));
System.arraycopy(lastTerm.bytes, 0, prefix.bytes, 0, prefixLength);
final long startFP = out.getFilePointer(); prefix.length = prefixLength;
final BytesRef prefix = new BytesRef(indexPrefixLength);
for(int m=0;m<indexPrefixLength;m++) {
prefix.bytes[m] = (byte) prevTerm.ints[m];
}
prefix.length = indexPrefixLength;
// Write block header: // 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) { // 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 // 1st pass: pack term suffix bytes into byte[] blob
// TODO: cutover to bulk int codec... simple64? // TODO: cutover to bulk int codec... simple64?
final boolean isLeafBlock; // We optimize the leaf block case (block has only terms), writing a more
if (lastBlockIndex < start) { // compact format in this case:
// This block definitely does not contain sub-blocks: boolean isLeafBlock = hasSubBlocks == false;
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;
}
final List<FST<BytesRef>> subIndices; final List<FST<BytesRef>> subIndices;
int termCount;
long[] longs = new long[longsSize];
boolean absolute = true; boolean absolute = true;
if (isLeafBlock) { if (isLeafBlock) {
// Only terms:
subIndices = null; subIndices = null;
for (PendingEntry ent : slice) { for (int i=start;i<end;i++) {
assert ent.isTerm; PendingEntry ent = pending.get(i);
assert ent.isTerm: "i=" + i;
PendingTerm term = (PendingTerm) ent; PendingTerm term = (PendingTerm) ent;
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
BlockTermState state = term.state; BlockTermState state = term.state;
final int suffix = term.term.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
// if (DEBUG) { /*
// BytesRef suffixBytes = new BytesRef(suffix); if (DEBUG) {
// System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix); BytesRef suffixBytes = new BytesRef(suffix);
// suffixBytes.length = suffix; System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
// System.out.println(" write term suffix=" + suffixBytes); suffixBytes.length = suffix;
// } System.out.println(" write term suffix=" + suffixBytes);
}
*/
// For leaf block we write suffix straight // For leaf block we write suffix straight
suffixWriter.writeVInt(suffix); 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: // Write term stats, to separate byte[] blob:
statsWriter.writeVInt(state.docFreq); statsWriter.writeVInt(state.docFreq);
@ -932,25 +762,29 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
bytesWriter.reset(); bytesWriter.reset();
absolute = false; absolute = false;
} }
termCount = length;
} else { } else {
// Mixed terms and sub-blocks:
subIndices = new ArrayList<>(); subIndices = new ArrayList<>();
termCount = 0; for (int i=start;i<end;i++) {
for (PendingEntry ent : slice) { PendingEntry ent = pending.get(i);
if (ent.isTerm) { if (ent.isTerm) {
PendingTerm term = (PendingTerm) ent; PendingTerm term = (PendingTerm) ent;
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
BlockTermState state = term.state; BlockTermState state = term.state;
final int suffix = term.term.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
// if (DEBUG) { /*
// BytesRef suffixBytes = new BytesRef(suffix); if (DEBUG) {
// System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix); BytesRef suffixBytes = new BytesRef(suffix);
// suffixBytes.length = suffix; System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
// System.out.println(" write term suffix=" + suffixBytes); suffixBytes.length = suffix;
// } System.out.println(" write term suffix=" + suffixBytes);
}
*/
// For non-leaf block we borrow 1 bit to record // For non-leaf block we borrow 1 bit to record
// if entry is term or sub-block // if entry is term or sub-block
suffixWriter.writeVInt(suffix<<1); 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: // Write term stats, to separate byte[] blob:
statsWriter.writeVInt(state.docFreq); statsWriter.writeVInt(state.docFreq);
@ -976,10 +810,9 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
bytesWriter.writeTo(metaWriter); bytesWriter.writeTo(metaWriter);
bytesWriter.reset(); bytesWriter.reset();
absolute = false; absolute = false;
termCount++;
} else { } else {
PendingBlock block = (PendingBlock) ent; PendingBlock block = (PendingBlock) ent;
assert StringHelper.startsWith(block.prefix, prefix);
final int suffix = block.prefix.length - prefixLength; final int suffix = block.prefix.length - prefixLength;
assert suffix > 0; assert suffix > 0;
@ -988,14 +821,19 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
// if entry is term or sub-block // if entry is term or sub-block
suffixWriter.writeVInt((suffix<<1)|1); suffixWriter.writeVInt((suffix<<1)|1);
suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix); suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix);
assert floorLeadLabel == -1 || (block.prefix.bytes[prefixLength] & 0xff) >= floorLeadLabel;
assert block.fp < startFP; assert block.fp < startFP;
// if (DEBUG) { /*
// BytesRef suffixBytes = new BytesRef(suffix); if (DEBUG) {
// System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix); BytesRef suffixBytes = new BytesRef(suffix);
// suffixBytes.length = suffix; System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
// System.out.println(" write sub-block suffix=" + toString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor); 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); suffixWriter.writeVLong(startFP - block.fp);
subIndices.add(block.index); subIndices.add(block.index);
@ -1024,48 +862,35 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
metaWriter.writeTo(out); metaWriter.writeTo(out);
metaWriter.reset(); metaWriter.reset();
// Remove slice replaced by block:
slice.clear();
if (lastBlockIndex >= start) {
if (lastBlockIndex < start+length) {
lastBlockIndex = start;
} else {
lastBlockIndex -= length;
}
}
// if (DEBUG) { // if (DEBUG) {
// System.out.println(" fpEnd=" + out.getFilePointer()); // 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) { TermsWriter(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo; this.fieldInfo = fieldInfo;
docsSeen = new FixedBitSet(maxDoc); 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.longsSize = postingsWriter.setField(fieldInfo);
this.longs = new long[longsSize];
} }
private final IntsRef scratchIntsRef = new IntsRef();
/** Writes one term's worth of postings. */ /** Writes one term's worth of postings. */
public void write(BytesRef text, TermsEnum termsEnum) throws IOException { public void write(BytesRef text, TermsEnum termsEnum) 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());
}
*/
BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen); BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen);
if (state != null) { if (state != null) {
@ -1073,18 +898,60 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter; assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter;
sumDocFreq += state.docFreq; sumDocFreq += state.docFreq;
sumTotalTermFreq += state.totalTermFreq; sumTotalTermFreq += state.totalTermFreq;
blockBuilder.add(Util.toIntsRef(text, scratchIntsRef), noOutputs.getNoOutput()); pushTerm(text);
PendingTerm term = new PendingTerm(BytesRef.deepCopyOf(text), state); PendingTerm term = new PendingTerm(text, state);
pending.add(term); pending.add(term);
numTerms++; numTerms++;
} }
} }
/** 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);
}
// Finishes all terms in this field // Finishes all terms in this field
public void finish(BytesRef minTerm, BytesRef maxTerm) throws IOException { public void finish(BytesRef minTerm, BytesRef maxTerm) throws IOException {
if (numTerms > 0) { 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: // We better have one final "root" block:
assert pending.size() == 1 && !pending.get(0).isTerm: "pending.size()=" + pending.size() + " pending=" + pending; assert pending.size() == 1 && !pending.get(0).isTerm: "pending.size()=" + pending.size() + " pending=" + pending;
@ -1097,13 +964,15 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
root.index.save(indexOut); root.index.save(indexOut);
//System.out.println(" write FST " + indexStartFP + " field=" + fieldInfo.name); //System.out.println(" write FST " + indexStartFP + " field=" + fieldInfo.name);
// if (SAVE_DOT_FILES || DEBUG) { /*
// final String dotFileName = segment + "_" + fieldInfo.name + ".dot"; if (DEBUG) {
// Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName)); final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
// Util.toDot(root.index, w, false, false); Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
// System.out.println("SAVED to " + dotFileName); Util.toDot(root.index, w, false, false);
// w.close(); System.out.println("SAVED to " + dotFileName);
// } w.close();
}
*/
fields.add(new FieldMetaData(fieldInfo, fields.add(new FieldMetaData(fieldInfo,
((PendingBlock) pending.get(0)).index.getEmptyOutput(), ((PendingBlock) pending.get(0)).index.getEmptyOutput(),

View File

@ -153,7 +153,8 @@ final class SegmentTermsEnumFrame {
entCount = code >>> 1; entCount = code >>> 1;
assert entCount > 0; assert entCount > 0;
isLastInFloor = (code & 1) != 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 // TODO: if suffixes were stored in random-access
// array structure, then we could do binary search // array structure, then we could do binary search

View File

@ -97,6 +97,31 @@ public abstract class StringHelper {
} }
} }
/**
* Returns <code>true</code> iff the ref starts with the given prefix.
* 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. * Returns <code>true</code> iff the ref starts with the given prefix.
* Otherwise <code>false</code>. * Otherwise <code>false</code>.

View File

@ -80,22 +80,14 @@ public class Builder<T> {
// current "frontier" // current "frontier"
private UnCompiledNode<T>[] 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 * Instantiates an FST/FSA builder without any pruning. A shortcut
* to {@link #Builder(FST.INPUT_TYPE, int, int, boolean, * 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. * boolean, int)} with pruning options turned off.
*/ */
public Builder(FST.INPUT_TYPE inputType, Outputs<T> outputs) { 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, public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs, boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
FreezeTail<T> freezeTail, boolean doPackFST, float acceptableOverheadRatio, boolean allowArrayArcs, boolean doPackFST, float acceptableOverheadRatio, boolean allowArrayArcs,
int bytesPageBits) { int bytesPageBits) {
this.minSuffixCount1 = minSuffixCount1; this.minSuffixCount1 = minSuffixCount1;
this.minSuffixCount2 = minSuffixCount2; this.minSuffixCount2 = minSuffixCount2;
this.freezeTail = freezeTail;
this.doShareNonSingletonNodes = doShareNonSingletonNodes; this.doShareNonSingletonNodes = doShareNonSingletonNodes;
this.shareMaxTailLength = shareMaxTailLength; this.shareMaxTailLength = shareMaxTailLength;
this.doPackFST = doPackFST; this.doPackFST = doPackFST;
@ -209,100 +200,95 @@ public class Builder<T> {
} }
private void freezeTail(int prefixLenPlus1) throws IOException { private void freezeTail(int prefixLenPlus1) throws IOException {
if (freezeTail != null) { //System.out.println(" compileTail " + prefixLenPlus1);
// Custom plugin: final int downTo = Math.max(1, prefixLenPlus1);
freezeTail.freeze(frontier, prefixLenPlus1, lastInput); for(int idx=lastInput.length; idx >= downTo; idx--) {
} else {
//System.out.println(" compileTail " + prefixLenPlus1);
final int downTo = Math.max(1, prefixLenPlus1);
for(int idx=lastInput.length; idx >= downTo; idx--) {
boolean doPrune = false; boolean doPrune = false;
boolean doCompile = false; boolean doCompile = false;
final UnCompiledNode<T> node = frontier[idx]; final UnCompiledNode<T> node = frontier[idx];
final UnCompiledNode<T> parent = frontier[idx-1]; 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; 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 { } else {
// if pruning is disabled (count is 0) we can always // my parent, about to be compiled, does make the cut, so
// compile current node // I'm definitely not pruned
doCompile = minSuffixCount2 == 0; 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); //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)) { if (node.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && node.inputCount == 1 && idx > 1)) {
// drop all arcs // drop all arcs
for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) { for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
@SuppressWarnings({"rawtypes","unchecked"}) final UnCompiledNode<T> target = @SuppressWarnings({"rawtypes","unchecked"}) final UnCompiledNode<T> target =
(UnCompiledNode<T>) node.arcs[arcIdx].target; (UnCompiledNode<T>) node.arcs[arcIdx].target;
target.clear(); target.clear();
}
node.numArcs = 0;
} }
node.numArcs = 0;
}
if (doPrune) { if (doPrune) {
// this node doesn't make it -- deref it // this node doesn't make it -- deref it
node.clear(); node.clear();
parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node); parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node);
} 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 { } else {
// replaceLast just to install
if (minSuffixCount2 != 0) { // nextFinalOutput/isFinal onto the arc
compileAllTargets(node, lastInput.length-idx); parent.replaceLast(lastInput.ints[lastInput.offset + idx-1],
} node,
final T nextFinalOutput = node.output; nextFinalOutput,
isFinal);
// We "fake" the node as being final if it has no // this node will stay in play for now, since we are
// outgoing arcs; in theory we could leave it // undecided on whether to prune it. later, it
// as non-final (the FST can represent this), but // will be either compiled or pruned, so we must
// FSTEnum, Util, etc., have trouble w/ non-final // allocate a new node:
// dead-end states: frontier[idx] = new UnCompiledNode<>(this, idx);
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);
}
} }
} }
} }

View File

@ -28,7 +28,6 @@ import org.apache.lucene.store.MMapDirectory;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef; import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.TimeUnits; import org.apache.lucene.util.TimeUnits;
import org.apache.lucene.util.packed.PackedInts; import org.apache.lucene.util.packed.PackedInts;
import org.junit.Ignore; import org.junit.Ignore;
@ -56,7 +55,7 @@ public class Test2BFST extends LuceneTestCase {
Outputs<Object> outputs = NoOutputs.getSingleton(); Outputs<Object> outputs = NoOutputs.getSingleton();
Object NO_OUTPUT = outputs.getNoOutput(); Object NO_OUTPUT = outputs.getNoOutput();
final Builder<Object> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, 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; int count = 0;
Random r = new Random(seed); Random r = new Random(seed);
@ -138,7 +137,7 @@ public class Test2BFST extends LuceneTestCase {
System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=bytes"); System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=bytes");
Outputs<BytesRef> outputs = ByteSequenceOutputs.getSingleton(); Outputs<BytesRef> outputs = ByteSequenceOutputs.getSingleton();
final Builder<BytesRef> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, 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]; byte[] outputBytes = new byte[20];
BytesRef output = new BytesRef(outputBytes); BytesRef output = new BytesRef(outputBytes);
@ -215,7 +214,7 @@ public class Test2BFST extends LuceneTestCase {
System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=long"); System.out.println("\nTEST: 3 GB size; doPack=" + doPack + " outputs=long");
Outputs<Long> outputs = PositiveIntOutputs.getSingleton(); Outputs<Long> outputs = PositiveIntOutputs.getSingleton();
final Builder<Long> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, 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; long output = 1;

View File

@ -326,7 +326,7 @@ public class TestFSTs extends LuceneTestCase {
final boolean doRewrite = random().nextBoolean(); 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(); boolean storeOrd = random().nextBoolean();
if (VERBOSE) { if (VERBOSE) {
@ -469,7 +469,7 @@ public class TestFSTs extends LuceneTestCase {
this.outputs = outputs; this.outputs = outputs;
this.doPack = doPack; 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; protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@ -1113,7 +1113,7 @@ public class TestFSTs extends LuceneTestCase {
public void testFinalOutputOnEndState() throws Exception { public void testFinalOutputOnEndState() throws Exception {
final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(); 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("stat", new IntsRef()), 17L);
builder.add(Util.toUTF32("station", new IntsRef()), 10L); builder.add(Util.toUTF32("station", new IntsRef()), 10L);
final FST<Long> fst = builder.finish(); final FST<Long> fst = builder.finish();
@ -1128,7 +1128,7 @@ public class TestFSTs extends LuceneTestCase {
public void testInternalFinalState() throws Exception { public void testInternalFinalState() throws Exception {
final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(); final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
final boolean willRewrite = random().nextBoolean(); 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("stat"), new IntsRef()), outputs.getNoOutput());
builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRef()), outputs.getNoOutput()); builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRef()), outputs.getNoOutput());
final FST<Long> fst = builder.finish(); final FST<Long> fst = builder.finish();

View File

@ -41,11 +41,11 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef; 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.Builder;
import org.apache.lucene.util.fst.ByteSequenceOutputs; import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.BytesRefFSTEnum; import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST; import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.NoOutputs;
import org.apache.lucene.util.fst.PairOutputs.Pair; import org.apache.lucene.util.fst.PairOutputs.Pair;
import org.apache.lucene.util.fst.PairOutputs; import org.apache.lucene.util.fst.PairOutputs;
import org.apache.lucene.util.fst.PositiveIntOutputs; import org.apache.lucene.util.fst.PositiveIntOutputs;
@ -92,8 +92,6 @@ import org.apache.lucene.util.packed.PackedInts;
public final class VersionBlockTreeTermsWriter extends FieldsConsumer { public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
// private static boolean DEBUG = IDVersionSegmentTermsEnum.DEBUG;
static final PairOutputs<BytesRef,Long> FST_OUTPUTS = new PairOutputs<>(ByteSequenceOutputs.getSingleton(), static final PairOutputs<BytesRef,Long> FST_OUTPUTS = new PairOutputs<>(ByteSequenceOutputs.getSingleton(),
PositiveIntOutputs.getSingleton()); PositiveIntOutputs.getSingleton());
@ -109,7 +107,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
* #VersionBlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */ * #VersionBlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int)}. */
public final static int DEFAULT_MAX_BLOCK_SIZE = 48; 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; //private final static boolean SAVE_DOT_FILES = false;
static final int OUTPUT_FLAGS_NUM_BITS = 2; static final int OUTPUT_FLAGS_NUM_BITS = 2;
@ -281,22 +279,42 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
} }
private static final class PendingTerm extends PendingEntry { private static final class PendingTerm extends PendingEntry {
public final BytesRef term; public final byte[] termBytes;
// stats + metadata // stats + metadata
public final BlockTermState state; public final BlockTermState state;
public PendingTerm(BytesRef term, BlockTermState state) { public PendingTerm(BytesRef term, BlockTermState state) {
super(true); super(true);
this.term = term; this.termBytes = new byte[term.length];
System.arraycopy(term.bytes, term.offset, termBytes, 0, term.length);
this.state = state; this.state = state;
} }
@Override @Override
public String toString() { 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 { private static final class PendingBlock extends PendingEntry {
public final BytesRef prefix; public final BytesRef prefix;
public final long fp; public final long fp;
@ -305,7 +323,6 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
public final boolean hasTerms; public final boolean hasTerms;
public final boolean isFloor; public final boolean isFloor;
public final int floorLeadByte; public final int floorLeadByte;
private final IntsRef scratchIntsRef = new IntsRef();
/** Max version for all terms in this block. */ /** Max version for all terms in this block. */
private final long maxVersion; private final long maxVersion;
@ -322,12 +339,13 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
@Override @Override
public String toString() { 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; assert scratchBytes.getFilePointer() == 0;
@ -338,9 +356,9 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
// outputs sharing in the FST // outputs sharing in the FST
scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor)); scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor));
if (isFloor) { if (isFloor) {
scratchBytes.writeVInt(floorBlocks.size()); scratchBytes.writeVInt(blocks.size()-1);
for (PendingBlock sub : floorBlocks) { for (int i=1;i<blocks.size();i++) {
assert sub.floorLeadByte != -1; PendingBlock sub = blocks.get(i);
maxVersionIndex = Math.max(maxVersionIndex, sub.maxVersion); maxVersionIndex = Math.max(maxVersionIndex, sub.maxVersion);
//if (DEBUG) { //if (DEBUG) {
// System.out.println(" write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff)); // System.out.println(" write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff));
@ -353,7 +371,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
final Builder<Pair<BytesRef,Long>> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1, final Builder<Pair<BytesRef,Long>> indexBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1,
0, 0, true, false, Integer.MAX_VALUE, 0, 0, true, false, Integer.MAX_VALUE,
FST_OUTPUTS, null, false, FST_OUTPUTS, false,
PackedInts.COMPACT, true, 15); PackedInts.COMPACT, true, 15);
//if (DEBUG) { //if (DEBUG) {
// System.out.println(" compile index for prefix=" + prefix); // System.out.println(" compile index for prefix=" + prefix);
@ -366,26 +384,18 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
scratchBytes.reset(); scratchBytes.reset();
// Copy over index for all sub-blocks // Copy over index for all sub-blocks
for(PendingBlock block : blocks) {
if (subIndices != null) { if (block.subIndices != null) {
for(FST<Pair<BytesRef,Long>> subIndex : subIndices) { for(FST<Pair<BytesRef,Long>> subIndex : block.subIndices) {
append(indexBuilder, subIndex); append(indexBuilder, subIndex, scratchIntsRef);
}
}
if (floorBlocks != null) {
for (PendingBlock sub : floorBlocks) {
if (sub.subIndices != null) {
for(FST<Pair<BytesRef,Long>> subIndex : sub.subIndices) {
append(indexBuilder, subIndex);
}
} }
sub.subIndices = null; block.subIndices = null;
} }
} }
index = indexBuilder.finish(); index = indexBuilder.finish();
subIndices = null;
assert subIndices == null;
/* /*
Writer w = new OutputStreamWriter(new FileOutputStream("out.dot")); Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
@ -398,7 +408,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
// TODO: maybe we could add bulk-add method to // TODO: maybe we could add bulk-add method to
// Builder? Takes FST and unions it w/ current // Builder? Takes FST and unions it w/ current
// FST. // FST.
private void append(Builder<Pair<BytesRef,Long>> builder, FST<Pair<BytesRef,Long>> subIndex) throws IOException { private void append(Builder<Pair<BytesRef,Long>> builder, FST<Pair<BytesRef,Long>> subIndex, IntsRef scratchIntsRef) throws IOException {
final BytesRefFSTEnum<Pair<BytesRef,Long>> subIndexEnum = new BytesRefFSTEnum<>(subIndex); final BytesRefFSTEnum<Pair<BytesRef,Long>> subIndexEnum = new BytesRefFSTEnum<>(subIndex);
BytesRefFSTEnum.InputOutput<Pair<BytesRef,Long>> indexEnt; BytesRefFSTEnum.InputOutput<Pair<BytesRef,Long>> indexEnt;
while((indexEnt = subIndexEnum.next()) != null) { while((indexEnt = subIndexEnum.next()) != null) {
@ -410,7 +420,8 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
} }
} }
final RAMOutputStream scratchBytes = new RAMOutputStream(); private final RAMOutputStream scratchBytes = new RAMOutputStream();
private final IntsRef scratchIntsRef = new IntsRef();
class TermsWriter { class TermsWriter {
private final FieldInfo fieldInfo; private final FieldInfo fieldInfo;
@ -419,384 +430,198 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
final FixedBitSet docsSeen; final FixedBitSet docsSeen;
long indexStartFP; long indexStartFP;
// Used only to partition terms into the block tree; we // Records index into pending where the current prefix at that
// don't pull an FST from this builder: // length "started"; for example, if current term starts with 't',
private final NoOutputs noOutputs; // startsByPrefix[0] is the index into pending for the first
private final Builder<Object> blockBuilder; // 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];
// PendingTerm or PendingBlock: 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<>(); private final List<PendingEntry> pending = new ArrayList<>();
// Index into pending of most recently written block // Reused in writeBlocks:
private int lastBlockIndex = -1; private final List<PendingBlock> newBlocks = new ArrayList<>();
// Re-used when segmenting a too-large block into floor /** Writes the top count entries in pending, using prevTerm to compute the prefix. */
// blocks: void writeBlocks(int prefixLength, int count) throws IOException {
private int[] subBytes = new int[10];
private int[] subTermCounts = new int[10];
private int[] subTermCountSums = new int[10];
private int[] subSubCounts = new int[10];
// This class assigns terms to blocks "naturally", ie, assert count > 0;
// according to the number of terms under a given prefix
// that we encounter:
private class FindBlocks extends Builder.FreezeTail<Object> {
@Override /*
public void freeze(final Builder.UnCompiledNode<Object>[] frontier, int prefixLenPlus1, final IntsRef lastInput) throws IOException { if (DEBUG) {
BytesRef br = new BytesRef(lastTerm.bytes);
//if (DEBUG) System.out.println(" freeze prefixLenPlus1=" + prefixLenPlus1); br.offset = lastTerm.offset;
br.length = prefixLength;
for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) { System.out.println("writeBlocks: " + br.utf8ToString() + " count=" + count);
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);
}
} }
} */
// Write the top count entries on the pending stack as // Root block better write all remaining pending entries:
// one or more blocks. Returns how many blocks were assert prefixLength > 0 || count == pending.size();
// written. 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 { int lastSuffixLeadLabel = -1;
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);
} 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 // True if we saw at least one term in this block (we record if a block
// in each block, to make floor blocks authoritative // 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;
/* int start = pending.size()-count;
if (DEBUG) { int end = pending.size();
final BytesRef prefix = new BytesRef(prefixLength); int nextBlockStart = start;
for(int m=0;m<prefixLength;m++) { int nextFloorLeadLabel = -1;
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=" + toString(prefix) + " " + prefix + " count=" + count + " pending.size()=" + pending.size());
}
*/
//System.out.println("\nwbs count=" + count);
final int savLabel = prevTerm.ints[prevTerm.offset + prefixLength]; for (int i=start; i<end; i++) {
// Count up how many items fall under PendingEntry ent = pending.get(i);
// 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;
for(PendingEntry ent : slice) { int suffixLeadLabel;
// First byte in the suffix of this term if (ent.isTerm) {
final int suffixLeadLabel; PendingTerm term = (PendingTerm) ent;
if (ent.isTerm) { if (term.termBytes.length == prefixLength) {
PendingTerm term = (PendingTerm) ent; // Suffix is 0, i.e. prefix 'foo' and term is
if (term.term.length == prefixLength) { // 'foo' so the term has empty string suffix
// Suffix is 0, ie prefix 'foo' and term is // in this block
// 'foo' so the term has empty string suffix assert lastSuffixLeadLabel == -1;
// in this block suffixLeadLabel = -1;
assert lastSuffixLeadLabel == -1;
assert numSubs == 0;
suffixLeadLabel = -1;
} else {
suffixLeadLabel = term.term.bytes[term.term.offset + prefixLength] & 0xff;
}
} else { } else {
PendingBlock block = (PendingBlock) ent; suffixLeadLabel = term.termBytes[prefixLength] & 0xff;
assert block.prefix.length > prefixLength;
suffixLeadLabel = block.prefix.bytes[block.prefix.offset + prefixLength] & 0xff;
} }
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++;
}
if (ent.isTerm) {
termCount++;
} else {
subCount++;
}
}
if (subBytes.length == numSubs) {
subBytes = ArrayUtil.grow(subBytes);
subTermCounts = ArrayUtil.grow(subTermCounts);
subSubCounts = ArrayUtil.grow(subSubCounts);
}
subBytes[numSubs] = lastSuffixLeadLabel;
subTermCounts[numSubs] = termCount;
subSubCounts[numSubs] = subCount;
numSubs++;
/*
if (lastSuffixLeadLabel == -1) {
System.out.println(" sub " + -1 + " termCount=" + termCount + " subCount=" + subCount);
} else { } else {
System.out.println(" sub " + Integer.toHexString(lastSuffixLeadLabel) + " termCount=" + termCount + " subCount=" + subCount); 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 (subTermCountSums.length < numSubs) { if (suffixLeadLabel != lastSuffixLeadLabel) {
subTermCountSums = ArrayUtil.grow(subTermCountSums, numSubs); 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));
// Roll up (backwards) the termCounts; postings impl hasTerms = false;
// needs this to know where to pull the term slice hasSubBlocks = false;
// from its pending terms stack: nextFloorLeadLabel = suffixLeadLabel;
int sum = 0; nextBlockStart = i;
for(int idx=numSubs-1;idx>=0;idx--) {
sum += subTermCounts[idx];
subTermCountSums[idx] = sum;
}
// 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(" = " + 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];
if (curStart == 0) {
break;
}
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;
}
} }
lastSuffixLeadLabel = suffixLeadLabel;
} }
prevTerm.ints[prevTerm.offset + prefixLength] = savLabel; if (ent.isTerm) {
hasTerms = true;
assert firstBlock != null; } else {
firstBlock.compileIndex(floorBlocks, scratchBytes); hasSubBlocks = true;
}
pending.add(firstBlock);
//if (DEBUG) System.out.println(" done pending.size()=" + pending.size());
} }
lastBlockIndex = pending.size()-1;
// 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));
}
assert newBlocks.isEmpty() == false;
PendingBlock firstBlock = newBlocks.get(0);
assert firstBlock.isFloor || newBlocks.size() == 1;
firstBlock.compileIndex(newBlocks, scratchBytes, scratchIntsRef);
// Remove slice from the top of the pending stack, that we just wrote:
pending.subList(pending.size()-count, pending.size()).clear();
// Append new block
pending.add(firstBlock);
newBlocks.clear();
} }
// for debugging /** Writes the specified slice (start is inclusive, end is exclusive)
@SuppressWarnings("unused") * from pending stack as a new block. If isFloor is true, there
private String toString(BytesRef b) { * were too many (more than maxItemsInBlock) entries sharing the
try { * same prefix, and so we broke it into multiple floor blocks where
return b.utf8ToString() + " " + b; * we record the starting label of the suffix of each floor block. */
} catch (Throwable t) { private PendingBlock writeBlock(int prefixLength, boolean isFloor, int floorLeadLabel, int start, int end, boolean hasTerms, boolean hasSubBlocks) throws IOException {
// 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();
}
}
// Writes all entries in the pending slice as a single assert end > start;
// block:
private PendingBlock writeBlock(IntsRef prevTerm, int prefixLength, int indexPrefixLength, int startBackwards, int length,
int futureTermCount, boolean isFloor, int floorLeadByte, boolean isLastInFloor) throws IOException {
assert length > 0; long startFP = out.getFilePointer();
final int start = pending.size()-startBackwards; // if (DEBUG) System.out.println(" writeBlock fp=" + startFP + " isFloor=" + isFloor + " floorLeadLabel=" + floorLeadLabel + " start=" + start + " end=" + end + " hasTerms=" + hasTerms + " hasSubBlocks=" + hasSubBlocks);
assert start >= 0: "pending.size()=" + pending.size() + " startBackwards=" + startBackwards + " length=" + length; boolean hasFloorLeadLabel = isFloor && floorLeadLabel != -1;
final List<PendingEntry> slice = pending.subList(start, start + length); final BytesRef prefix = new BytesRef(prefixLength + (hasFloorLeadLabel ? 1 : 0));
System.arraycopy(lastTerm.bytes, 0, prefix.bytes, 0, prefixLength);
final long startFP = out.getFilePointer(); prefix.length = prefixLength;
final BytesRef prefix = new BytesRef(indexPrefixLength);
for(int m=0;m<indexPrefixLength;m++) {
prefix.bytes[m] = (byte) prevTerm.ints[m];
}
prefix.length = indexPrefixLength;
// Write block header: // 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) { // 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 // 1st pass: pack term suffix bytes into byte[] blob
// TODO: cutover to bulk int codec... simple64? // TODO: cutover to bulk int codec... simple64?
final boolean isLeafBlock; // We optimize the leaf block case (block has only terms), writing a more
if (lastBlockIndex < start) { // compact format in this case:
// This block definitely does not contain sub-blocks: boolean isLeafBlock = hasSubBlocks == false;
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;
}
final List<FST<Pair<BytesRef,Long>>> subIndices; final List<FST<Pair<BytesRef,Long>>> subIndices;
int termCount;
long[] longs = new long[longsSize];
boolean absolute = true; boolean absolute = true;
long maxVersionInBlock = -1; long maxVersionInBlock = -1;
// int countx = 0;
if (isLeafBlock) { if (isLeafBlock) {
// Only terms:
subIndices = null; subIndices = null;
for (PendingEntry ent : slice) { for (int i=start;i<end;i++) {
assert ent.isTerm; PendingEntry ent = pending.get(i);
assert ent.isTerm: "i=" + i;
PendingTerm term = (PendingTerm) ent; PendingTerm term = (PendingTerm) ent;
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
BlockTermState state = term.state; BlockTermState state = term.state;
maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion); maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion);
final int suffix = term.term.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
// if (DEBUG) { /*
// BytesRef suffixBytes = new BytesRef(suffix); if (DEBUG) {
// System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix); BytesRef suffixBytes = new BytesRef(suffix);
// suffixBytes.length = suffix; System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
// System.out.println(" " + (countx++) + ": write term suffix=" + toString(suffixBytes)); suffixBytes.length = suffix;
// } System.out.println(" write term suffix=" + suffixBytes);
}
*/
// For leaf block we write suffix straight // For leaf block we write suffix straight
suffixWriter.writeVInt(suffix); 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 meta data // Write term meta data
postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute); postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
@ -808,26 +633,30 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
bytesWriter.reset(); bytesWriter.reset();
absolute = false; absolute = false;
} }
termCount = length;
} else { } else {
// Mixed terms and sub-blocks:
subIndices = new ArrayList<>(); subIndices = new ArrayList<>();
termCount = 0; for (int i=start;i<end;i++) {
for (PendingEntry ent : slice) { PendingEntry ent = pending.get(i);
if (ent.isTerm) { if (ent.isTerm) {
PendingTerm term = (PendingTerm) ent; PendingTerm term = (PendingTerm) ent;
assert StringHelper.startsWith(term.termBytes, prefix): "term.term=" + term.termBytes + " prefix=" + prefix;
BlockTermState state = term.state; BlockTermState state = term.state;
maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion); maxVersionInBlock = Math.max(maxVersionInBlock, ((IDVersionTermState) state).idVersion);
final int suffix = term.term.length - prefixLength; final int suffix = term.termBytes.length - prefixLength;
// if (DEBUG) { /*
// BytesRef suffixBytes = new BytesRef(suffix); if (DEBUG) {
// System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix); BytesRef suffixBytes = new BytesRef(suffix);
// suffixBytes.length = suffix; System.arraycopy(term.term.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
// System.out.println(" " + (countx++) + ": write term suffix=" + toString(suffixBytes)); suffixBytes.length = suffix;
// } System.out.println(" write term suffix=" + suffixBytes);
}
*/
// For non-leaf block we borrow 1 bit to record // For non-leaf block we borrow 1 bit to record
// if entry is term or sub-block // if entry is term or sub-block
suffixWriter.writeVInt(suffix<<1); 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;
// TODO: now that terms dict "sees" these longs, // TODO: now that terms dict "sees" these longs,
// we can explore better column-stride encodings // we can explore better column-stride encodings
@ -846,11 +675,10 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
bytesWriter.writeTo(metaWriter); bytesWriter.writeTo(metaWriter);
bytesWriter.reset(); bytesWriter.reset();
absolute = false; absolute = false;
termCount++;
} else { } else {
PendingBlock block = (PendingBlock) ent; PendingBlock block = (PendingBlock) ent;
maxVersionInBlock = Math.max(maxVersionInBlock, block.maxVersion); maxVersionInBlock = Math.max(maxVersionInBlock, block.maxVersion);
assert StringHelper.startsWith(block.prefix, prefix);
final int suffix = block.prefix.length - prefixLength; final int suffix = block.prefix.length - prefixLength;
assert suffix > 0; assert suffix > 0;
@ -859,14 +687,19 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
// if entry is term or sub-block // if entry is term or sub-block
suffixWriter.writeVInt((suffix<<1)|1); suffixWriter.writeVInt((suffix<<1)|1);
suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix); suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix);
assert floorLeadLabel == -1 || (block.prefix.bytes[prefixLength] & 0xff) >= floorLeadLabel;
assert block.fp < startFP; assert block.fp < startFP;
// if (DEBUG) { /*
// BytesRef suffixBytes = new BytesRef(suffix); if (DEBUG) {
// System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix); BytesRef suffixBytes = new BytesRef(suffix);
// suffixBytes.length = suffix; System.arraycopy(block.prefix.bytes, prefixLength, suffixBytes.bytes, 0, suffix);
// System.out.println(" " + (countx++) + ": write sub-block suffix=" + toString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor); 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); suffixWriter.writeVLong(startFP - block.fp);
subIndices.add(block.index); subIndices.add(block.index);
@ -890,46 +723,26 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
metaWriter.writeTo(out); metaWriter.writeTo(out);
metaWriter.reset(); metaWriter.reset();
// Remove slice replaced by block:
slice.clear();
if (lastBlockIndex >= start) {
if (lastBlockIndex < start+length) {
lastBlockIndex = start;
} else {
lastBlockIndex -= length;
}
}
// if (DEBUG) { // if (DEBUG) {
// System.out.println(" fpEnd=" + out.getFilePointer()); // System.out.println(" fpEnd=" + out.getFilePointer());
// } // }
return new PendingBlock(prefix, maxVersionInBlock, 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, maxVersionInBlock, startFP, hasTerms, isFloor, floorLeadLabel, subIndices);
} }
TermsWriter(FieldInfo fieldInfo) { TermsWriter(FieldInfo fieldInfo) {
this.fieldInfo = fieldInfo; this.fieldInfo = fieldInfo;
docsSeen = new FixedBitSet(maxDoc); 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.longsSize = postingsWriter.setField(fieldInfo);
this.longs = new long[longsSize];
} }
private final IntsRef scratchIntsRef = new IntsRef();
/** Writes one term's worth of postings. */ /** Writes one term's worth of postings. */
public void write(BytesRef text, TermsEnum termsEnum) throws IOException { public void write(BytesRef text, TermsEnum termsEnum) throws IOException {
@ -938,7 +751,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
if (state != null && ((IDVersionPostingsWriter) postingsWriter).lastDocID != -1) { if (state != null && ((IDVersionPostingsWriter) postingsWriter).lastDocID != -1) {
assert state.docFreq != 0; assert state.docFreq != 0;
assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter; assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY || state.totalTermFreq >= state.docFreq: "postingsWriter=" + postingsWriter;
blockBuilder.add(Util.toIntsRef(text, scratchIntsRef), noOutputs.getNoOutput()); pushTerm(text);
PendingTerm term = new PendingTerm(BytesRef.deepCopyOf(text), state); PendingTerm term = new PendingTerm(BytesRef.deepCopyOf(text), state);
pending.add(term); pending.add(term);
@ -946,10 +759,51 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
} }
} }
/** 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);
}
// Finishes all terms in this field // Finishes all terms in this field
public void finish(BytesRef minTerm, BytesRef maxTerm) throws IOException { public void finish(BytesRef minTerm, BytesRef maxTerm) throws IOException {
if (numTerms > 0) { if (numTerms > 0) {
blockBuilder.finish();
// 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: // We better have one final "root" block:
assert pending.size() == 1 && !pending.get(0).isTerm: "pending.size()=" + pending.size() + " pending=" + pending; assert pending.size() == 1 && !pending.get(0).isTerm: "pending.size()=" + pending.size() + " pending=" + pending;

View File

@ -239,7 +239,7 @@ public class FSTCompletionBuilder {
final Object empty = outputs.getNoOutput(); final Object empty = outputs.getNoOutput();
final Builder<Object> builder = new Builder<>( final Builder<Object> builder = new Builder<>(
FST.INPUT_TYPE.BYTE1, 0, 0, true, true, FST.INPUT_TYPE.BYTE1, 0, 0, true, true,
shareMaxTailLength, outputs, null, false, shareMaxTailLength, outputs, false,
PackedInts.DEFAULT, true, 15); PackedInts.DEFAULT, true, 15);
BytesRef scratch = new BytesRef(); BytesRef scratch = new BytesRef();

View File

@ -288,7 +288,6 @@ public class FSTTester<T> {
allowRandomSuffixSharing ? random.nextBoolean() : true, allowRandomSuffixSharing ? random.nextBoolean() : true,
allowRandomSuffixSharing ? TestUtil.nextInt(random, 1, 10) : Integer.MAX_VALUE, allowRandomSuffixSharing ? TestUtil.nextInt(random, 1, 10) : Integer.MAX_VALUE,
outputs, outputs,
null,
willRewrite, willRewrite,
PackedInts.DEFAULT, PackedInts.DEFAULT,
true, true,