-fix: actually write a new segments_N file, removing any problematic segments
@@ -1161,6 +1191,7 @@ public class CheckIndex {
public static void main(String[] args) throws IOException, InterruptedException {
boolean doFix = false;
+ boolean verbose = false;
List onlySegments = new ArrayList();
String indexPath = null;
int i = 0;
@@ -1168,6 +1199,9 @@ public class CheckIndex {
if (args[i].equals("-fix")) {
doFix = true;
i++;
+ } else if (args[i].equals("-verbose")) {
+ verbose = true;
+ i++;
} else if (args[i].equals("-segment")) {
if (i == args.length-1) {
System.out.println("ERROR: missing name for -segment option");
@@ -1190,6 +1224,7 @@ public class CheckIndex {
System.out.println("\nUsage: java org.apache.lucene.index.CheckIndex pathToIndex [-fix] [-segment X] [-segment Y]\n" +
"\n" +
" -fix: actually write a new segments_N file, removing any problematic segments\n" +
+ " -verbose: print additional details\n" +
" -segment X: only check the specified segments. This can be specified multiple\n" +
" times, to check more than one segment, eg '-segment _2 -segment _a'.\n" +
" You can't use this with the -fix option\n" +
@@ -1231,7 +1266,7 @@ public class CheckIndex {
}
CheckIndex checker = new CheckIndex(dir);
- checker.setInfoStream(System.out);
+ checker.setInfoStream(System.out, verbose);
Status result = checker.checkIndex(onlySegments);
if (result.missingSegments) {
diff --git a/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
index dae350aae3a..fc99a719202 100644
--- a/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
+++ b/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
@@ -26,6 +26,7 @@ import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.IOUtils;
final class FreqProxTermsWriter extends TermsHashConsumer {
@@ -58,6 +59,8 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
+ boolean success = false;
+
try {
TermsHash termsHash = null;
@@ -100,8 +103,9 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
if (termsHash != null) {
termsHash.reset();
}
+ success = true;
} finally {
- consumer.close();
+ IOUtils.closeSafely(!success, consumer);
}
}
diff --git a/lucene/src/java/org/apache/lucene/index/MultiTerms.java b/lucene/src/java/org/apache/lucene/index/MultiTerms.java
index 2785170dfb3..168a142dd65 100644
--- a/lucene/src/java/org/apache/lucene/index/MultiTerms.java
+++ b/lucene/src/java/org/apache/lucene/index/MultiTerms.java
@@ -17,13 +17,15 @@ package org.apache.lucene.index;
* limitations under the License.
*/
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ReaderUtil;
-import java.io.IOException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Comparator;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* Exposes flex API, merged from flex API of
@@ -58,6 +60,23 @@ public final class MultiTerms extends Terms {
termComp = _termComp;
}
+ @Override
+ public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+ final List termsEnums = new ArrayList();
+ for(int i=0;i 0) {
+ return new MultiTermsEnum(subSlices).reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
+ } else {
+ return TermsEnum.EMPTY;
+ }
+ }
+
@Override
public TermsEnum iterator() throws IOException {
diff --git a/lucene/src/java/org/apache/lucene/index/SegmentMerger.java b/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
index 52159ab316b..96908747a35 100644
--- a/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
+++ b/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
@@ -562,12 +562,14 @@ final class SegmentMerger {
}
codec = segmentWriteState.segmentCodecs.codec();
final FieldsConsumer consumer = codec.fieldsConsumer(segmentWriteState);
+ boolean success = false;
try {
consumer.merge(mergeState,
new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
+ success = true;
} finally {
- consumer.close();
+ IOUtils.closeSafely(!success, consumer);
}
}
diff --git a/lucene/src/java/org/apache/lucene/index/Terms.java b/lucene/src/java/org/apache/lucene/index/Terms.java
index b7a27e0d0c9..6cdf1f353db 100644
--- a/lucene/src/java/org/apache/lucene/index/Terms.java
+++ b/lucene/src/java/org/apache/lucene/index/Terms.java
@@ -19,9 +19,11 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Comparator;
+
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
/**
* Access to the terms in a specific field. See {@link Fields}.
@@ -37,7 +39,40 @@ public abstract class Terms {
/** Returns an iterator that will step through all
* terms. This method will not return null.*/
public abstract TermsEnum iterator() throws IOException;
-
+
+ /** Returns a TermsEnum that iterates over all terms that
+ * are accepted by the provided {@link
+ * CompiledAutomaton}. If the startTerm is
+ * provided then the returned enum will only accept terms
+ * > startTerm, but you still must call
+ * next() first to get to the first term. Note that the
+ * provided startTerm must be accepted by
+ * the automaton.
+ *
+ *
NOTE: the returned TermsEnum cannot
+ * seek
. */
+ public TermsEnum intersect(CompiledAutomaton compiled, final BytesRef startTerm) throws IOException {
+ // TODO: eventually we could support seekCeil/Exact on
+ // the returned enum, instead of only being able to seek
+ // at the start
+ if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+ throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
+ }
+ if (startTerm == null) {
+ return new AutomatonTermsEnum(iterator(), compiled);
+ } else {
+ return new AutomatonTermsEnum(iterator(), compiled) {
+ @Override
+ protected BytesRef nextSeekTerm(BytesRef term) throws IOException {
+ if (term == null) {
+ term = startTerm;
+ }
+ return super.nextSeekTerm(term);
+ }
+ };
+ }
+ }
+
/** Return the BytesRef Comparator used to sort terms
* provided by the iterator. This method may return null
* if there are no terms. This method may be invoked
diff --git a/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java b/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java
index 36e24c2ec25..2a070f260c7 100644
--- a/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java
+++ b/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java
@@ -29,10 +29,8 @@ public class BlockTermState extends OrdTermState {
public int docFreq; // how many docs have this term
public long totalTermFreq; // total number of occurrences of this term
- public int termCount; // term ord are in the current block
- public long blockFilePointer; // fp into the terms dict primary file (_X.tib) that holds this term
-
- public int blockTermCount; // how many terms in current block
+ public int termBlockOrd; // the term's ord in the current block
+ public long blockFilePointer; // fp into the terms dict primary file (_X.tim) that holds this term
@Override
public void copyFrom(TermState _other) {
@@ -41,7 +39,7 @@ public class BlockTermState extends OrdTermState {
super.copyFrom(_other);
docFreq = other.docFreq;
totalTermFreq = other.totalTermFreq;
- termCount = other.termCount;
+ termBlockOrd = other.termBlockOrd;
blockFilePointer = other.blockFilePointer;
// NOTE: don't copy blockTermCount;
@@ -51,6 +49,6 @@ public class BlockTermState extends OrdTermState {
@Override
public String toString() {
- return "ord=" + ord + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termCount=" + termCount + " blockFP=" + blockFilePointer;
+ return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer;
}
}
diff --git a/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java b/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
index c827926e90c..ffd685c7c2e 100644
--- a/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
+++ b/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
@@ -106,7 +106,7 @@ public class BlockTermsReader extends FieldsProducer {
}
}
- //private String segment;
+ // private String segment;
public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, IOContext context,
int termsCacheSize, int codecId)
@@ -115,7 +115,7 @@ public class BlockTermsReader extends FieldsProducer {
this.postingsReader = postingsReader;
termsCache = new DoubleBarrelLRUCache(termsCacheSize);
- //this.segment = segment;
+ // this.segment = segment;
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
context);
@@ -321,6 +321,9 @@ public class BlockTermsReader extends FieldsProducer {
/* Common prefix used for all terms in this block. */
private int termBlockPrefix;
+ /* How many terms in current block */
+ private int blockTermCount;
+
private byte[] docFreqBytes;
private final ByteArrayDataInput freqReader = new ByteArrayDataInput();
private int metaDataUpto;
@@ -358,16 +361,14 @@ public class BlockTermsReader extends FieldsProducer {
throw new IllegalStateException("terms index was not loaded");
}
- /*
- System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this=" + this);
+ //System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this=" + this);
if (didIndexNext) {
if (nextIndexTerm == null) {
- System.out.println(" nextIndexTerm=null");
+ //System.out.println(" nextIndexTerm=null");
} else {
- System.out.println(" nextIndexTerm=" + nextIndexTerm.utf8ToString());
+ //System.out.println(" nextIndexTerm=" + nextIndexTerm.utf8ToString());
}
}
- */
// Check cache
if (useCache) {
@@ -444,7 +445,7 @@ public class BlockTermsReader extends FieldsProducer {
//System.out.println(" seek: term=" + term.utf8ToString());
} else {
//System.out.println(" skip seek");
- if (state.termCount == state.blockTermCount && !nextBlock()) {
+ if (state.termBlockOrd == blockTermCount && !nextBlock()) {
indexIsCurrent = false;
return SeekStatus.END;
}
@@ -480,9 +481,9 @@ public class BlockTermsReader extends FieldsProducer {
// but it could be in next block. We
// must scan to end-of-block to set common
// prefix for next block:
- if (state.termCount < state.blockTermCount) {
- while(state.termCount < state.blockTermCount-1) {
- state.termCount++;
+ if (state.termBlockOrd < blockTermCount) {
+ while(state.termBlockOrd < blockTermCount-1) {
+ state.termBlockOrd++;
state.ord++;
termSuffixesReader.skipBytes(termSuffixesReader.readVInt());
}
@@ -505,7 +506,7 @@ public class BlockTermsReader extends FieldsProducer {
// Target's prefix is before the common prefix
// of this block, so we position to start of
// block and return NOT_FOUND:
- assert state.termCount == 0;
+ assert state.termBlockOrd == 0;
final int suffix = termSuffixesReader.readVInt();
term.length = termBlockPrefix + suffix;
@@ -523,7 +524,7 @@ public class BlockTermsReader extends FieldsProducer {
// Test every term in this block
while (true) {
- state.termCount++;
+ state.termBlockOrd++;
state.ord++;
final int suffix = termSuffixesReader.readVInt();
@@ -581,7 +582,7 @@ public class BlockTermsReader extends FieldsProducer {
}
}
- if (state.termCount == state.blockTermCount) {
+ if (state.termBlockOrd == blockTermCount) {
// Must pre-fill term for next block's common prefix
term.length = termBlockPrefix + suffix;
if (term.bytes.length < term.length) {
@@ -613,7 +614,7 @@ public class BlockTermsReader extends FieldsProducer {
@Override
public BytesRef next() throws IOException {
- //System.out.println("BTR.next() seekPending=" + seekPending + " pendingSeekCount=" + state.termCount);
+ //System.out.println("BTR.next() seekPending=" + seekPending + " pendingSeekCount=" + state.termBlockOrd);
// If seek was previously called and the term was cached,
// usually caller is just going to pull a D/&PEnum or get
@@ -623,7 +624,7 @@ public class BlockTermsReader extends FieldsProducer {
if (seekPending) {
assert !indexIsCurrent;
in.seek(state.blockFilePointer);
- final int pendingSeekCount = state.termCount;
+ final int pendingSeekCount = state.termBlockOrd;
boolean result = nextBlock();
final long savOrd = state.ord;
@@ -633,7 +634,7 @@ public class BlockTermsReader extends FieldsProducer {
// on a real term:
assert result;
- while(state.termCount < pendingSeekCount) {
+ while(state.termBlockOrd < pendingSeekCount) {
BytesRef nextResult = _next();
assert nextResult != null;
}
@@ -647,8 +648,8 @@ public class BlockTermsReader extends FieldsProducer {
metadata, ie docFreq, totalTermFreq or pulls a D/&PEnum, we then (lazily)
decode all metadata up to the current term. */
private BytesRef _next() throws IOException {
- //System.out.println("BTR._next seg=" + segment + " this=" + this + " termCount=" + state.termCount + " (vs " + state.blockTermCount + ")");
- if (state.termCount == state.blockTermCount && !nextBlock()) {
+ //System.out.println("BTR._next seg=" + segment + " this=" + this + " termCount=" + state.termBlockOrd + " (vs " + blockTermCount + ")");
+ if (state.termBlockOrd == blockTermCount && !nextBlock()) {
//System.out.println(" eof");
indexIsCurrent = false;
return null;
@@ -663,12 +664,12 @@ public class BlockTermsReader extends FieldsProducer {
term.grow(term.length);
}
termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
- state.termCount++;
+ state.termBlockOrd++;
// NOTE: meaningless in the non-ord case
state.ord++;
- //System.out.println(" return term=" + fieldInfo.name + ":" + term.utf8ToString() + " " + term);
+ //System.out.println(" return term=" + fieldInfo.name + ":" + term.utf8ToString() + " " + term + " tbOrd=" + state.termBlockOrd);
return term;
}
@@ -695,9 +696,10 @@ public class BlockTermsReader extends FieldsProducer {
public DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
//System.out.println("BTR.docs this=" + this);
decodeMetaData();
- //System.out.println(" state.docFreq=" + state.docFreq);
+ //System.out.println("BTR.docs: state.docFreq=" + state.docFreq);
final DocsEnum docsEnum = postingsReader.docs(fieldInfo, state, liveDocs, reuse);
assert docsEnum != null;
+ //System.out.println("BTR.docs: return docsEnum=" + docsEnum);
return docsEnum;
}
@@ -716,7 +718,7 @@ public class BlockTermsReader extends FieldsProducer {
@Override
public void seekExact(BytesRef target, TermState otherState) throws IOException {
- //System.out.println("BTR.seek termState target=" + target.utf8ToString() + " " + target + " this=" + this);
+ //System.out.println("BTR.seekExact termState target=" + target.utf8ToString() + " " + target + " this=" + this);
assert otherState != null && otherState instanceof BlockTermState;
assert !doOrd || ((BlockTermState) otherState).ord < numTerms;
state.copyFrom(otherState);
@@ -800,9 +802,9 @@ public class BlockTermsReader extends FieldsProducer {
//System.out.println("BTR.nextBlock() fp=" + in.getFilePointer() + " this=" + this);
state.blockFilePointer = in.getFilePointer();
- state.blockTermCount = in.readVInt();
- //System.out.println(" blockTermCount=" + state.blockTermCount);
- if (state.blockTermCount == 0) {
+ blockTermCount = in.readVInt();
+ //System.out.println(" blockTermCount=" + blockTermCount);
+ if (blockTermCount == 0) {
return false;
}
termBlockPrefix = in.readVInt();
@@ -826,7 +828,7 @@ public class BlockTermsReader extends FieldsProducer {
freqReader.reset(docFreqBytes, 0, len);
metaDataUpto = 0;
- state.termCount = 0;
+ state.termBlockOrd = 0;
postingsReader.readTermsBlock(in, fieldInfo, state);
@@ -838,7 +840,7 @@ public class BlockTermsReader extends FieldsProducer {
}
private void decodeMetaData() throws IOException {
- //System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termCount + " state=" + state);
+ //System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termBlockOrd + " state=" + state);
if (!seekPending) {
// TODO: cutover to random-access API
// here.... really stupid that we have to decode N
@@ -846,10 +848,10 @@ public class BlockTermsReader extends FieldsProducer {
// that we really need...
// lazily catch up on metadata decode:
- final int limit = state.termCount;
+ final int limit = state.termBlockOrd;
// We must set/incr state.termCount because
// postings impl can look at this
- state.termCount = metaDataUpto;
+ state.termBlockOrd = metaDataUpto;
// TODO: better API would be "jump straight to term=N"???
while (metaDataUpto < limit) {
//System.out.println(" decode mdUpto=" + metaDataUpto);
@@ -870,9 +872,9 @@ public class BlockTermsReader extends FieldsProducer {
postingsReader.nextTerm(fieldInfo, state);
metaDataUpto++;
- state.termCount++;
+ state.termBlockOrd++;
}
- //} else {
+ } else {
//System.out.println(" skip! seekPending");
}
}
diff --git a/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java b/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
index ff1af7ba040..950af269d76 100644
--- a/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
+++ b/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
@@ -66,7 +66,7 @@ public class BlockTermsWriter extends FieldsConsumer {
private final TermsIndexWriterBase termsIndexWriter;
private final List fields = new ArrayList();
- //private final String segment;
+ // private final String segment;
public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter,
SegmentWriteState state, PostingsWriterBase postingsWriter)
@@ -80,7 +80,7 @@ public class BlockTermsWriter extends FieldsConsumer {
writeHeader(out);
currentField = null;
this.postingsWriter = postingsWriter;
- //segment = state.segmentName;
+ // segment = state.segmentName;
//System.out.println("BTW.init seg=" + state.segmentName);
@@ -188,7 +188,7 @@ public class BlockTermsWriter extends FieldsConsumer {
@Override
public PostingsConsumer startTerm(BytesRef text) throws IOException {
- //System.out.println("BTW.startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
+ //System.out.println("BTW: startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
postingsWriter.startTerm();
return postingsWriter;
}
@@ -199,7 +199,7 @@ public class BlockTermsWriter extends FieldsConsumer {
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
assert stats.docFreq > 0;
- //System.out.println("BTW.finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
+ //System.out.println("BTW: finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, stats);
@@ -308,7 +308,7 @@ public class BlockTermsWriter extends FieldsConsumer {
bytesWriter.writeTo(out);
bytesWriter.reset();
- postingsWriter.flushTermsBlock();
+ postingsWriter.flushTermsBlock(pendingCount, pendingCount);
lastPrevTerm.copy(pendingTerms[pendingCount-1].term);
pendingCount = 0;
}
diff --git a/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java b/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
new file mode 100644
index 00000000000..850a39e6710
--- /dev/null
+++ b/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
@@ -0,0 +1,2832 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.ByteArrayOutputStream;
+import java.io.Closeable;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.Writer;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.TreeMap;
+
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.FieldsEnum;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.RunAutomaton;
+import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.fst.ByteSequenceOutputs;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.Outputs;
+import org.apache.lucene.util.fst.Util;
+
+/** A block-based terms index and dictionary that assigns
+ * terms to variable length blocks according to how they
+ * share prefixes. The terms index is a prefix trie
+ * whose leaves are term blocks. The advantage of this
+ * approach is that {@link #seekExact} is often able to
+ * determine a term cannot exist without doing any IO, and
+ * intersection with Automata is very fast. Note that this
+ * terms dictionary has it's own fixed terms index (ie, it
+ * does not support a pluggable terms index
+ * implementation).
+ *
+ *
NOTE: this terms dictionary does not support
+ * index divisor when opening an IndexReader. Instead, you
+ * can change the min/maxItemsPerBlock during indexing.
+ *
+ *
The data structure used by this implementation is very
+ * similar to a burst trie
+ * (http://citeseer.ist.psu.edu/viewdoc/summary?doi=10.1.1.18.3499),
+ * but with added logic to break up too-large blocks of all
+ * terms sharing a given prefix into smaller ones.
+ *
+ *
Use {@link CheckIndex} with the -verbose
+ * option to see summary statistics on the blocks in the
+ * dictionary.
+ *
+ * See {@link BlockTreeTermsWriter}.
+ *
+ * @lucene.experimental
+ */
+
+public class BlockTreeTermsReader extends FieldsProducer {
+
+ // Open input to the main terms dict file (_X.tib)
+ private final IndexInput in;
+
+ //private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+
+ // Reads the terms dict entries, to gather state to
+ // produce DocsEnum on demand
+ private final PostingsReaderBase postingsReader;
+
+ private final TreeMap fields = new TreeMap();
+
+ // keeps the dirStart offset
+ protected long dirOffset;
+ protected long indexDirOffset;
+
+ private String segment;
+
+ public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, String segment,
+ PostingsReaderBase postingsReader, IOContext ioContext,
+ int codecId, int indexDivisor)
+ throws IOException {
+
+ this.postingsReader = postingsReader;
+
+ this.segment = segment;
+ in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTreeTermsWriter.TERMS_EXTENSION),
+ ioContext);
+
+ boolean success = false;
+ IndexInput indexIn = null;
+
+ try {
+ readHeader(in);
+ if (indexDivisor != -1) {
+ indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
+ ioContext);
+ readIndexHeader(indexIn);
+ }
+
+ // Have PostingsReader init itself
+ postingsReader.init(in);
+
+ // Read per-field details
+ seekDir(in, dirOffset);
+ if (indexDivisor != -1) {
+ seekDir(indexIn, indexDirOffset);
+ }
+
+ final int numFields = in.readVInt();
+
+ for(int i=0;i= 0;
+ final int numBytes = in.readVInt();
+ final BytesRef rootCode = new BytesRef(new byte[numBytes]);
+ in.readBytes(rootCode.bytes, 0, numBytes);
+ rootCode.length = numBytes;
+ final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+ assert fieldInfo != null: "field=" + field;
+ final long sumTotalTermFreq = fieldInfo.indexOptions == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
+ final long sumDocFreq = in.readVLong();
+ final long indexStartFP = indexDivisor != -1 ? indexIn.readVLong() : 0;
+ assert !fields.containsKey(fieldInfo.name);
+ fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, indexStartFP, indexIn));
+ }
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeSafely(true, indexIn, this);
+ } else if (indexDivisor != -1) {
+ indexIn.close();
+ }
+ }
+ }
+
+ protected void readHeader(IndexInput input) throws IOException {
+ CodecUtil.checkHeader(input, BlockTreeTermsWriter.CODEC_NAME,
+ BlockTreeTermsWriter.VERSION_START,
+ BlockTreeTermsWriter.VERSION_CURRENT);
+ dirOffset = input.readLong();
+ }
+
+ protected void readIndexHeader(IndexInput input) throws IOException {
+ CodecUtil.checkHeader(input, BlockTreeTermsWriter.CODEC_NAME,
+ BlockTreeTermsWriter.VERSION_START,
+ BlockTreeTermsWriter.VERSION_CURRENT);
+ indexDirOffset = input.readLong();
+ }
+
+ protected void seekDir(IndexInput input, long dirOffset)
+ throws IOException {
+ input.seek(dirOffset);
+ }
+
+ // for debugging
+ // private static String toHex(int v) {
+ // return "0x" + Integer.toHexString(v);
+ // }
+
+ @Override
+ public void close() throws IOException {
+ try {
+ IOUtils.closeSafely(false, in, postingsReader);
+ } finally {
+ for(FieldReader field : fields.values()) {
+ field.close();
+ }
+ // Clear so refs to terms index is GCable even if
+ // app hangs onto us:
+ fields.clear();
+ }
+ }
+
+ public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection files) {
+ files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, BlockTreeTermsWriter.TERMS_EXTENSION));
+ files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION));
+ }
+
+ public static void getExtensions(Collection extensions) {
+ extensions.add(BlockTreeTermsWriter.TERMS_EXTENSION);
+ extensions.add(BlockTreeTermsWriter.TERMS_INDEX_EXTENSION);
+ }
+
+ @Override
+ public FieldsEnum iterator() {
+ return new TermFieldsEnum();
+ }
+
+ @Override
+ public Terms terms(String field) throws IOException {
+ return fields.get(field);
+ }
+
+ // Iterates through all fields
+ private class TermFieldsEnum extends FieldsEnum {
+ final Iterator it;
+ FieldReader current;
+
+ TermFieldsEnum() {
+ it = fields.values().iterator();
+ }
+
+ @Override
+ public String next() {
+ if (it.hasNext()) {
+ current = it.next();
+ return current.fieldInfo.name;
+ } else {
+ current = null;
+ return null;
+ }
+ }
+
+ @Override
+ public TermsEnum terms() throws IOException {
+ return current.iterator();
+ }
+ }
+
+ // for debugging
+ String brToString(BytesRef b) {
+ if (b == null) {
+ return "null";
+ } else {
+ try {
+ return b.utf8ToString() + " " + b;
+ } catch (Throwable t) {
+ // If BytesRef isn't actually UTF8, or it's eg a
+ // prefix of UTF8 that ends mid-unicode-char, we
+ // fallback to hex:
+ return b.toString();
+ }
+ }
+ }
+
+ public static class Stats {
+ public int indexNodeCount;
+ public int indexArcCount;
+ public int indexNumBytes;
+
+ public long totalTermCount;
+ public long totalTermBytes;
+
+
+ public int nonFloorBlockCount;
+ public int floorBlockCount;
+ public int floorSubBlockCount;
+ public int mixedBlockCount;
+ public int termsOnlyBlockCount;
+ public int subBlocksOnlyBlockCount;
+ public int totalBlockCount;
+
+ public int[] blockCountByPrefixLen = new int[10];
+ private int startBlockCount;
+ private int endBlockCount;
+ public long totalBlockSuffixBytes;
+ public long totalBlockStatsBytes;
+
+ // Postings impl plus the other few vInts stored in
+ // the frame:
+ public long totalBlockOtherBytes;
+
+ public final String segment;
+ public final String field;
+
+ public Stats(String segment, String field) {
+ this.segment = segment;
+ this.field = field;
+ }
+
+ void startBlock(FieldReader.SegmentTermsEnum.Frame frame, boolean isFloor) {
+ totalBlockCount++;
+ if (isFloor) {
+ if (frame.fp == frame.fpOrig) {
+ floorBlockCount++;
+ }
+ floorSubBlockCount++;
+ } else {
+ nonFloorBlockCount++;
+ }
+
+ if (blockCountByPrefixLen.length <= frame.prefix) {
+ blockCountByPrefixLen = ArrayUtil.grow(blockCountByPrefixLen, 1+frame.prefix);
+ }
+ blockCountByPrefixLen[frame.prefix]++;
+ startBlockCount++;
+ totalBlockSuffixBytes += frame.suffixesReader.length();
+ totalBlockStatsBytes += frame.statsReader.length();
+ }
+
+ void endBlock(FieldReader.SegmentTermsEnum.Frame frame) {
+ final int termCount = frame.isLeafBlock ? frame.entCount : frame.state.termBlockOrd;
+ final int subBlockCount = frame.entCount - termCount;
+ totalTermCount += termCount;
+ if (termCount != 0 && subBlockCount != 0) {
+ mixedBlockCount++;
+ } else if (termCount != 0) {
+ termsOnlyBlockCount++;
+ } else if (subBlockCount != 0) {
+ subBlocksOnlyBlockCount++;
+ } else {
+ throw new IllegalStateException();
+ }
+ endBlockCount++;
+ final long otherBytes = frame.fpEnd - frame.fp - frame.suffixesReader.length() - frame.statsReader.length();
+ assert otherBytes > 0 : "otherBytes=" + otherBytes + " frame.fp=" + frame.fp + " frame.fpEnd=" + frame.fpEnd;
+ totalBlockOtherBytes += otherBytes;
+ }
+
+ void term(BytesRef term) {
+ totalTermBytes += term.length;
+ }
+
+ void finish() {
+ assert startBlockCount == endBlockCount: "startBlockCount=" + startBlockCount + " endBlockCount=" + endBlockCount;
+ assert totalBlockCount == floorSubBlockCount + nonFloorBlockCount: "floorSubBlockCount=" + floorSubBlockCount + " nonFloorBlockCount=" + nonFloorBlockCount + " totalBlockCount=" + totalBlockCount;
+ assert totalBlockCount == mixedBlockCount + termsOnlyBlockCount + subBlocksOnlyBlockCount: "totalBlockCount=" + totalBlockCount + " mixedBlockCount=" + mixedBlockCount + " subBlocksOnlyBlockCount=" + subBlocksOnlyBlockCount + " termsOnlyBlockCount=" + termsOnlyBlockCount;
+ }
+
+ @Override
+ public String toString() {
+ final ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
+ final PrintStream out = new PrintStream(bos);
+
+ out.println(" index FST:");
+ out.println(" " + indexNodeCount + " nodes");
+ out.println(" " + indexArcCount + " arcs");
+ out.println(" " + indexNumBytes + " bytes");
+ out.println(" terms:");
+ out.println(" " + totalTermCount + " terms");
+ out.println(" " + totalTermBytes + " bytes" + (totalTermCount != 0 ? " (" + String.format("%.1f", ((double) totalTermBytes)/totalTermCount) + " bytes/term)" : ""));
+ out.println(" blocks:");
+ out.println(" " + totalBlockCount + " blocks");
+ out.println(" " + termsOnlyBlockCount + " terms-only blocks");
+ out.println(" " + subBlocksOnlyBlockCount + " sub-block-only blocks");
+ out.println(" " + mixedBlockCount + " mixed blocks");
+ out.println(" " + floorBlockCount + " floor blocks");
+ out.println(" " + (totalBlockCount-floorSubBlockCount) + " non-floor blocks");
+ out.println(" " + floorSubBlockCount + " floor sub-blocks");
+ out.println(" " + totalBlockSuffixBytes + " term suffix bytes" + (totalBlockCount != 0 ? " (" + String.format("%.1f", ((double) totalBlockSuffixBytes)/totalBlockCount) + " suffix-bytes/block)" : ""));
+ out.println(" " + totalBlockStatsBytes + " term stats bytes" + (totalBlockCount != 0 ? " (" + String.format("%.1f", ((double) totalBlockStatsBytes)/totalBlockCount) + " stats-bytes/block)" : ""));
+ out.println(" " + totalBlockOtherBytes + " other bytes" + (totalBlockCount != 0 ? " (" + String.format("%.1f", ((double) totalBlockOtherBytes)/totalBlockCount) + " other-bytes/block)" : ""));
+ if (totalBlockCount != 0) {
+ out.println(" by prefix length:");
+ int total = 0;
+ for(int prefix=0;prefix fstOutputs = ByteSequenceOutputs.getSingleton();
+ final BytesRef NO_OUTPUT = fstOutputs.getNoOutput();
+
+ public final class FieldReader extends Terms implements Closeable {
+ final long numTerms;
+ final FieldInfo fieldInfo;
+ final long sumTotalTermFreq;
+ final long sumDocFreq;
+ final long indexStartFP;
+ final long rootBlockFP;
+ final BytesRef rootCode;
+ private FST index;
+
+ //private boolean DEBUG;
+
+ FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, long indexStartFP, IndexInput indexIn) throws IOException {
+ assert numTerms > 0;
+ this.fieldInfo = fieldInfo;
+ //DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
+ this.numTerms = numTerms;
+ this.sumTotalTermFreq = sumTotalTermFreq;
+ this.sumDocFreq = sumDocFreq;
+ this.indexStartFP = indexStartFP;
+ this.rootCode = rootCode;
+ // if (DEBUG) {
+ // System.out.println("BTTR: seg=" + segment + " field=" + fieldInfo.name + " rootBlockCode=" + rootCode + " divisor=" + indexDivisor);
+ // }
+
+ rootBlockFP = (new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong() >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
+
+ if (indexIn != null) {
+ final IndexInput clone = (IndexInput) indexIn.clone();
+ //System.out.println("start=" + indexStartFP + " field=" + fieldInfo.name);
+ clone.seek(indexStartFP);
+ index = new FST(clone, ByteSequenceOutputs.getSingleton());
+
+ /*
+ if (false) {
+ final String dotFileName = segment + "_" + fieldInfo.name + ".dot";
+ Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName));
+ Util.toDot(index, w, false, false);
+ System.out.println("FST INDEX: SAVED to " + dotFileName);
+ w.close();
+ }
+ */
+ }
+ }
+
+ /** For debugging -- used by CheckIndex too*/
+ // TODO: maybe push this into Terms?
+ public Stats computeStats() throws IOException {
+ return new SegmentTermsEnum().computeBlockStats();
+ }
+
+ @Override
+ public Comparator getComparator() {
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
+ }
+
+ @Override
+ public void close() {
+ super.close();
+ }
+
+ @Override
+ public TermsEnum iterator() throws IOException {
+ return new SegmentTermsEnum();
+ }
+
+ @Override
+ public long getUniqueTermCount() {
+ return numTerms;
+ }
+
+ @Override
+ public long getSumTotalTermFreq() {
+ return sumTotalTermFreq;
+ }
+
+ @Override
+ public long getSumDocFreq() {
+ return sumDocFreq;
+ }
+
+ @Override
+ public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+ if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+ throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
+ }
+ return new IntersectEnum(compiled, startTerm);
+ }
+
+ // NOTE: cannot seek!
+ private final class IntersectEnum extends TermsEnum {
+ private final IndexInput in;
+
+ private Frame[] stack;
+
+ @SuppressWarnings("unchecked") private FST.Arc[] arcs = new FST.Arc[5];
+
+ private final RunAutomaton runAutomaton;
+ private final CompiledAutomaton compiledAutomaton;
+
+ private Frame currentFrame;
+
+ private final BytesRef term = new BytesRef();
+
+ // TODO: can we share this with the frame in STE?
+ private final class Frame {
+ final int ord;
+ long fp;
+ long fpOrig;
+ long fpEnd;
+ long lastSubFP;
+
+ // State in automaton
+ int state;
+
+ int metaDataUpto;
+
+ byte[] suffixBytes = new byte[128];
+ final ByteArrayDataInput suffixesReader = new ByteArrayDataInput();
+
+ byte[] statBytes = new byte[64];
+ final ByteArrayDataInput statsReader = new ByteArrayDataInput();
+
+ byte[] floorData = new byte[32];
+ final ByteArrayDataInput floorDataReader = new ByteArrayDataInput();
+
+ // Length of prefix shared by all terms in this block
+ int prefix;
+
+ // Number of entries (term or sub-block) in this block
+ int entCount;
+
+ // Which term we will next read
+ int nextEnt;
+
+ // True if this block is either not a floor block,
+ // or, it's the last sub-block of a floor block
+ boolean isLastInFloor;
+
+ // True if all entries are terms
+ boolean isLeafBlock;
+
+ int numFollowFloorBlocks;
+ int nextFloorLabel;
+
+ Transition[] transitions;
+ int curTransitionMax;
+ int transitionIndex;
+
+ FST.Arc arc;
+
+ final BlockTermState termState;
+
+ // Cumulative output so far
+ BytesRef outputPrefix;
+
+ private int startBytePos;
+ private int suffix;
+
+ public Frame(int ord) throws IOException {
+ this.ord = ord;
+ termState = postingsReader.newTermState();
+ termState.totalTermFreq = -1;
+ }
+
+ void loadNextFloorBlock() throws IOException {
+ assert numFollowFloorBlocks > 0;
+ //if (DEBUG) System.out.println(" loadNextFoorBlock trans=" + transitions[transitionIndex]);
+
+ do {
+ fp = fpOrig + (floorDataReader.readVLong() >>> 1);
+ numFollowFloorBlocks--;
+ // if (DEBUG) System.out.println(" skip floor block2! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[transitionIndex].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+ if (numFollowFloorBlocks != 0) {
+ nextFloorLabel = floorDataReader.readByte() & 0xff;
+ } else {
+ nextFloorLabel = 256;
+ }
+ // if (DEBUG) System.out.println(" nextFloorLabel=" + (char) nextFloorLabel);
+ } while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[transitionIndex].getMin());
+
+ load(null);
+ }
+
+ public void setState(int state) {
+ this.state = state;
+ transitionIndex = 0;
+ transitions = compiledAutomaton.sortedTransitions[state];
+ if (transitions.length != 0) {
+ curTransitionMax = transitions[0].getMax();
+ } else {
+ curTransitionMax = -1;
+ }
+ }
+
+ void load(BytesRef frameIndexData) throws IOException {
+
+ // if (DEBUG) System.out.println(" load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state));
+
+ if (frameIndexData != null && transitions.length != 0) {
+ // Floor frame
+ if (floorData.length < frameIndexData.length) {
+ this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
+ }
+ System.arraycopy(frameIndexData.bytes, frameIndexData.offset, floorData, 0, frameIndexData.length);
+ floorDataReader.reset(floorData, 0, frameIndexData.length);
+ // Skip first long -- has redundant fp, hasTerms
+ // flag, isFloor flag
+ final long code = floorDataReader.readVLong();
+ if ((code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) {
+ numFollowFloorBlocks = floorDataReader.readVInt();
+ nextFloorLabel = floorDataReader.readByte() & 0xff;
+ // if (DEBUG) System.out.println(" numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
+
+ // If current state is accept, we must process
+ // first block in case it has empty suffix:
+ if (!runAutomaton.isAccept(state)) {
+ // Maybe skip floor blocks:
+ while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[0].getMin()) {
+ fp = fpOrig + (floorDataReader.readVLong() >>> 1);
+ numFollowFloorBlocks--;
+ // if (DEBUG) System.out.println(" skip floor block! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+ if (numFollowFloorBlocks != 0) {
+ nextFloorLabel = floorDataReader.readByte() & 0xff;
+ } else {
+ nextFloorLabel = 256;
+ }
+ }
+ }
+ }
+ }
+
+ in.seek(fp);
+ int code = in.readVInt();
+ entCount = code >>> 1;
+ assert entCount > 0;
+ isLastInFloor = (code & 1) != 0;
+
+ // term suffixes:
+ code = in.readVInt();
+ isLeafBlock = (code & 1) != 0;
+ int numBytes = code >>> 1;
+ // if (DEBUG) System.out.println(" entCount=" + entCount + " lastInFloor?=" + isLastInFloor + " leafBlock?=" + isLeafBlock + " numSuffixBytes=" + numBytes);
+ if (suffixBytes.length < numBytes) {
+ suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+ }
+ in.readBytes(suffixBytes, 0, numBytes);
+ suffixesReader.reset(suffixBytes, 0, numBytes);
+
+ // stats
+ numBytes = in.readVInt();
+ if (statBytes.length < numBytes) {
+ statBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+ }
+ in.readBytes(statBytes, 0, numBytes);
+ statsReader.reset(statBytes, 0, numBytes);
+ metaDataUpto = 0;
+
+ termState.termBlockOrd = 0;
+ nextEnt = 0;
+
+ postingsReader.readTermsBlock(in, fieldInfo, termState);
+
+ if (!isLastInFloor) {
+ // Sub-blocks of a single floor block are always
+ // written one after another -- tail recurse:
+ fpEnd = in.getFilePointer();
+ }
+ }
+
+ // TODO: maybe add scanToLabel; should give perf boost
+
+ public boolean next() {
+ return isLeafBlock ? nextLeaf() : nextNonLeaf();
+ }
+
+ // Decodes next entry; returns true if it's a sub-block
+ public boolean nextLeaf() {
+ //if (DEBUG) System.out.println(" frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+ assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+ nextEnt++;
+ suffix = suffixesReader.readVInt();
+ startBytePos = suffixesReader.getPosition();
+ suffixesReader.skipBytes(suffix);
+ return false;
+ }
+
+ public boolean nextNonLeaf() {
+ //if (DEBUG) System.out.println(" frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+ assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+ nextEnt++;
+ final int code = suffixesReader.readVInt();
+ suffix = code >>> 1;
+ startBytePos = suffixesReader.getPosition();
+ suffixesReader.skipBytes(suffix);
+ if ((code & 1) == 0) {
+ // A normal term
+ termState.termBlockOrd++;
+ return false;
+ } else {
+ // A sub-block; make sub-FP absolute:
+ lastSubFP = fp - suffixesReader.readVLong();
+ return true;
+ }
+ }
+
+ public int getTermBlockOrd() {
+ return isLeafBlock ? nextEnt : termState.termBlockOrd;
+ }
+
+ public void decodeMetaData() throws IOException {
+
+ // lazily catch up on metadata decode:
+ final int limit = getTermBlockOrd();
+ assert limit > 0;
+
+ // We must set/incr state.termCount because
+ // postings impl can look at this
+ termState.termBlockOrd = metaDataUpto;
+
+ // TODO: better API would be "jump straight to term=N"???
+ while (metaDataUpto < limit) {
+
+ // TODO: we could make "tiers" of metadata, ie,
+ // decode docFreq/totalTF but don't decode postings
+ // metadata; this way caller could get
+ // docFreq/totalTF w/o paying decode cost for
+ // postings
+
+ // TODO: if docFreq were bulk decoded we could
+ // just skipN here:
+ termState.docFreq = statsReader.readVInt();
+ //if (DEBUG) System.out.println(" dF=" + state.docFreq);
+ if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
+ termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
+ //if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq);
+ }
+
+ postingsReader.nextTerm(fieldInfo, termState);
+ metaDataUpto++;
+ termState.termBlockOrd++;
+ }
+ }
+ }
+
+ private final BytesRef savedStartTerm;
+
+ // TODO: in some cases we can filter by length? eg
+ // regexp foo*bar must be at least length 6 bytes
+ public IntersectEnum(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+ // if (DEBUG) {
+ // System.out.println("\nintEnum.init seg=" + segment + " commonSuffix=" + brToString(compiled.commonSuffixRef));
+ // }
+ runAutomaton = compiled.runAutomaton;
+ compiledAutomaton = compiled;
+ in = (IndexInput) BlockTreeTermsReader.this.in.clone();
+ stack = new Frame[5];
+ for(int idx=0;idx();
+ }
+
+ // TODO: if the automaon is "smallish" we really
+ // should use the terms index to seek at least to
+ // the initial term and likely to subsequent terms
+ // (or, maybe just fallback to ATE for such cases).
+ // Else the seek cost of loading the frames will be
+ // too costly.
+
+ final FST.Arc arc = index.getFirstArc(arcs[0]);
+ // Empty string prefix must have an output in the index!
+ assert arc.isFinal();
+
+ // Special pushFrame since it's the first one:
+ final Frame f = stack[0];
+ f.fp = f.fpOrig = rootBlockFP;
+ f.prefix = 0;
+ f.setState(runAutomaton.getInitialState());
+ f.arc = arc;
+ f.outputPrefix = arc.output;
+ f.load(rootCode);
+
+ // for assert:
+ savedStartTerm = startTerm == null ? null : new BytesRef(startTerm);
+
+ currentFrame = f;
+ if (startTerm != null) {
+ seekToStartTerm(startTerm);
+ }
+ }
+
+ @Override
+ public TermState termState() throws IOException {
+ currentFrame.decodeMetaData();
+ return (TermState) currentFrame.termState.clone();
+ }
+
+ private Frame getFrame(int ord) throws IOException {
+ if (ord >= stack.length) {
+ final Frame[] next = new Frame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+ System.arraycopy(stack, 0, next, 0, stack.length);
+ for(int stackOrd=stack.length;stackOrd getArc(int ord) {
+ if (ord >= arcs.length) {
+ @SuppressWarnings("unchecked") final FST.Arc[] next = new FST.Arc[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+ System.arraycopy(arcs, 0, next, 0, arcs.length);
+ for(int arcOrd=arcs.length;arcOrd();
+ }
+ arcs = next;
+ }
+ return arcs[ord];
+ }
+
+ private Frame pushFrame(int state) throws IOException {
+ final Frame f = getFrame(currentFrame == null ? 0 : 1+currentFrame.ord);
+
+ f.fp = f.fpOrig = currentFrame.lastSubFP;
+ f.prefix = currentFrame.prefix + currentFrame.suffix;
+ // if (DEBUG) System.out.println(" pushFrame state=" + state + " prefix=" + f.prefix);
+ f.setState(state);
+
+ // Walk the arc through the index -- we only
+ // "bother" with this so we can get the floor data
+ // from the index and skip floor blocks when
+ // possible:
+ FST.Arc arc = currentFrame.arc;
+ int idx = currentFrame.prefix;
+ assert currentFrame.suffix > 0;
+ BytesRef output = currentFrame.outputPrefix;
+ while (idx < f.prefix) {
+ final int target = term.bytes[idx] & 0xff;
+ // TODO: we could be more efficient for the next()
+ // case by using current arc as starting point,
+ // passed to findTargetArc
+ arc = index.findTargetArc(target, arc, getArc(1+idx));
+ assert arc != null;
+ output = fstOutputs.add(output, arc.output);
+ idx++;
+ }
+
+ f.arc = arc;
+ f.outputPrefix = output;
+ assert arc.isFinal();
+ f.load(fstOutputs.add(output, arc.nextFinalOutput));
+ return f;
+ }
+
+ @Override
+ public BytesRef term() throws IOException {
+ return term;
+ }
+
+ @Override
+ public int docFreq() throws IOException {
+ //if (DEBUG) System.out.println("BTIR.docFreq");
+ currentFrame.decodeMetaData();
+ //if (DEBUG) System.out.println(" return " + currentFrame.termState.docFreq);
+ return currentFrame.termState.docFreq;
+ }
+
+ @Override
+ public long totalTermFreq() throws IOException {
+ currentFrame.decodeMetaData();
+ return currentFrame.termState.totalTermFreq;
+ }
+
+ @Override
+ public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
+ currentFrame.decodeMetaData();
+ return postingsReader.docs(fieldInfo, currentFrame.termState, skipDocs, reuse);
+ }
+
+ @Override
+ public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+ if (fieldInfo.indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+ return null;
+ } else {
+ currentFrame.decodeMetaData();
+ return postingsReader.docsAndPositions(fieldInfo, currentFrame.termState, skipDocs, reuse);
+ }
+ }
+
+ private int getState() {
+ int state = currentFrame.state;
+ for(int idx=0;idx 0) {
+ // A prefix of the common suffix overlaps with
+ // the suffix of the block prefix so we first
+ // test whether the prefix part matches:
+ final byte[] termBytes = term.bytes;
+ int termBytesPos = currentFrame.prefix - lenInPrefix;
+ assert termBytesPos >= 0;
+ final int termBytesPosEnd = currentFrame.prefix;
+ while (termBytesPos < termBytesPosEnd) {
+ if (termBytes[termBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
+ // if (DEBUG) {
+ // System.out.println(" skip: common suffix mismatch (in prefix)");
+ // }
+ continue nextTerm;
+ }
+ }
+ suffixBytesPos = currentFrame.startBytePos;
+ } else {
+ suffixBytesPos = currentFrame.startBytePos + currentFrame.suffix - compiledAutomaton.commonSuffixRef.length;
+ }
+
+ // Test overlapping suffix part:
+ final int commonSuffixBytesPosEnd = compiledAutomaton.commonSuffixRef.length;
+ while (commonSuffixBytesPos < commonSuffixBytesPosEnd) {
+ if (suffixBytes[suffixBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
+ // if (DEBUG) {
+ // System.out.println(" skip: common suffix mismatch");
+ // }
+ continue nextTerm;
+ }
+ }
+ }
+
+ // TODO: maybe we should do the same linear test
+ // that AutomatonTermsEnum does, so that if we
+ // reach a part of the automaton where .* is
+ // "temporarily" accepted, we just blindly .next()
+ // until the limit
+
+ // See if the term prefix matches the automaton:
+ int state = currentFrame.state;
+ for (int idx=0;idx[] arcs = new FST.Arc[5];
+
+ public SegmentTermsEnum() throws IOException {
+ //if (DEBUG) System.out.println("BTTR.init seg=" + segment);
+ in = (IndexInput) BlockTreeTermsReader.this.in.clone();
+ stack = new Frame[5];
+ for(int stackOrd=0;stackOrd();
+ }
+
+ // Init w/ root block; don't use index since it may
+ // not (and need not) have been loaded
+ //final FST.Arc arc = index.getFirstArc(arcs[0]);
+
+ // Empty string prefix must have an output in the index!
+ //assert arc.isFinal();
+
+ currentFrame = staticFrame;
+ final FST.Arc arc;
+ if (index != null) {
+ arc = index.getFirstArc(arcs[0]);
+ // Empty string prefix must have an output in the index!
+ assert arc.isFinal();
+ } else {
+ arc = null;
+ }
+ currentFrame = pushFrame(arc, rootCode, 0);
+ currentFrame.loadBlock();
+ validIndexPrefix = 0;
+ // if (DEBUG) {
+ // System.out.println("init frame state " + currentFrame.ord);
+ // printSeekState();
+ // }
+
+ //System.out.println();
+ // computeBlockStats().print(System.out);
+ }
+
+ /** Runs next() through the entire terms dict,
+ * computing aggregate statistics. */
+ public Stats computeBlockStats() throws IOException {
+
+ Stats stats = new Stats(segment, fieldInfo.name);
+ if (index != null) {
+ stats.indexNodeCount = index.getNodeCount();
+ stats.indexArcCount = index.getArcCount();
+ stats.indexNumBytes = index.sizeInBytes();
+ }
+
+ currentFrame = staticFrame;
+ FST.Arc arc;
+ if (index != null) {
+ arc = index.getFirstArc(arcs[0]);
+ // Empty string prefix must have an output in the index!
+ assert arc.isFinal();
+ } else {
+ arc = null;
+ }
+
+ // Empty string prefix must have an output in the
+ // index!
+ currentFrame = pushFrame(arc, rootCode, 0);
+ currentFrame.fpOrig = currentFrame.fp;
+ currentFrame.loadBlock();
+ validIndexPrefix = 0;
+
+ stats.startBlock(currentFrame, !currentFrame.isLastInFloor);
+
+ allTerms:
+ while (true) {
+
+ // Pop finished blocks
+ while (currentFrame.nextEnt == currentFrame.entCount) {
+ stats.endBlock(currentFrame);
+ if (!currentFrame.isLastInFloor) {
+ currentFrame.loadNextFloorBlock();
+ stats.startBlock(currentFrame, true);
+ } else {
+ if (currentFrame.ord == 0) {
+ break allTerms;
+ }
+ final long lastFP = currentFrame.fpOrig;
+ currentFrame = stack[currentFrame.ord-1];
+ assert lastFP == currentFrame.lastSubFP;
+ // if (DEBUG) {
+ // System.out.println(" reset validIndexPrefix=" + validIndexPrefix);
+ // }
+ }
+ }
+
+ while(true) {
+ if (currentFrame.next()) {
+ // Push to new block:
+ currentFrame = pushFrame(null, currentFrame.lastSubFP, term.length);
+ currentFrame.fpOrig = currentFrame.fp;
+ // This is a "next" frame -- even if it's
+ // floor'd we must pretend it isn't so we don't
+ // try to scan to the right floor frame:
+ currentFrame.isFloor = false;
+ //currentFrame.hasTerms = true;
+ currentFrame.loadBlock();
+ stats.startBlock(currentFrame, !currentFrame.isLastInFloor);
+ } else {
+ stats.term(term);
+ break;
+ }
+ }
+ }
+
+ stats.finish();
+
+ // Put root frame back:
+ currentFrame = staticFrame;
+ if (index != null) {
+ arc = index.getFirstArc(arcs[0]);
+ // Empty string prefix must have an output in the index!
+ assert arc.isFinal();
+ } else {
+ arc = null;
+ }
+ currentFrame = pushFrame(arc, rootCode, 0);
+ currentFrame.rewind();
+ currentFrame.loadBlock();
+ validIndexPrefix = 0;
+ term.length = 0;
+
+ return stats;
+ }
+
+ private Frame getFrame(int ord) throws IOException {
+ if (ord >= stack.length) {
+ final Frame[] next = new Frame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+ System.arraycopy(stack, 0, next, 0, stack.length);
+ for(int stackOrd=stack.length;stackOrd getArc(int ord) {
+ if (ord >= arcs.length) {
+ @SuppressWarnings("unchecked") final FST.Arc[] next = new FST.Arc[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
+ System.arraycopy(arcs, 0, next, 0, arcs.length);
+ for(int arcOrd=arcs.length;arcOrd();
+ }
+ arcs = next;
+ }
+ return arcs[ord];
+ }
+
+ @Override
+ public Comparator getComparator() {
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
+ }
+
+ // Pushes a frame we seek'd to
+ Frame pushFrame(FST.Arc arc, BytesRef frameData, int length) throws IOException {
+ scratchReader.reset(frameData.bytes, frameData.offset, frameData.length);
+ final long code = scratchReader.readVLong();
+ final long fpSeek = code >>> BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
+ final Frame f = getFrame(1+currentFrame.ord);
+ f.hasTerms = (code & BlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0;
+ f.hasTermsOrig = f.hasTerms;
+ f.isFloor = (code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0;
+ if (f.isFloor) {
+ f.setFloorData(scratchReader, frameData);
+ }
+ pushFrame(arc, fpSeek, length);
+
+ return f;
+ }
+
+ // Pushes next'd frame or seek'd frame; we later
+ // lazy-load the frame only when needed
+ Frame pushFrame(FST.Arc arc, long fp, int length) throws IOException {
+ final Frame f = getFrame(1+currentFrame.ord);
+ f.arc = arc;
+ if (f.fpOrig == fp && f.nextEnt != -1) {
+ //if (DEBUG) System.out.println(" push reused frame ord=" + f.ord + " fp=" + f.fp + " isFloor?=" + f.isFloor + " hasTerms=" + f.hasTerms + " pref=" + term + " nextEnt=" + f.nextEnt + " targetBeforeCurrentLength=" + targetBeforeCurrentLength + " term.length=" + term.length + " vs prefix=" + f.prefix);
+ if (f.prefix > targetBeforeCurrentLength) {
+ f.rewind();
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" skip rewind!");
+ // }
+ }
+ assert length == f.prefix;
+ } else {
+ f.nextEnt = -1;
+ f.prefix = length;
+ f.state.termBlockOrd = 0;
+ f.fpOrig = f.fp = fp;
+ f.lastSubFP = -1;
+ // if (DEBUG) {
+ // final int sav = term.length;
+ // term.length = length;
+ // System.out.println(" push new frame ord=" + f.ord + " fp=" + f.fp + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " pref=" + brToString(term));
+ // term.length = sav;
+ // }
+ }
+
+ return f;
+ }
+
+ // asserts only
+ private boolean clearEOF() {
+ eof = false;
+ return true;
+ }
+
+ // asserts only
+ private boolean setEOF() {
+ eof = true;
+ return true;
+ }
+
+ @Override
+ public boolean seekExact(final BytesRef target, final boolean useCache) throws IOException {
+
+ if (index == null) {
+ throw new IllegalStateException("terms index was not loaded");
+ }
+
+ if (term.bytes.length <= target.length) {
+ term.bytes = ArrayUtil.grow(term.bytes, 1+target.length);
+ }
+
+ assert clearEOF();
+
+ // if (DEBUG) {
+ // System.out.println("\nBTTR.seekExact seg=" + segment + " target=" + fieldInfo.name + ":" + brToString(target) + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix=" + validIndexPrefix);
+ // printSeekState();
+ // }
+
+ FST.Arc arc;
+ int targetUpto;
+ BytesRef output;
+
+ targetBeforeCurrentLength = currentFrame.ord;
+
+ if (currentFrame != staticFrame) {
+
+ // We are already seek'd; find the common
+ // prefix of new seek term vs current term and
+ // re-use the corresponding seek state. For
+ // example, if app first seeks to foobar, then
+ // seeks to foobaz, we can re-use the seek state
+ // for the first 5 bytes.
+
+ // if (DEBUG) {
+ // System.out.println(" re-use current seek state validIndexPrefix=" + validIndexPrefix);
+ // }
+
+ arc = arcs[0];
+ assert arc.isFinal();
+ output = arc.output;
+ targetUpto = 0;
+
+ Frame lastFrame = stack[0];
+ assert validIndexPrefix <= term.length;
+
+ final int targetLimit = Math.min(target.length, validIndexPrefix);
+
+ int cmp = 0;
+
+ // TODO: reverse vLong byte order for better FST
+ // prefix output sharing
+
+ // First compare up to valid seek frames:
+ while (targetUpto < targetLimit) {
+ cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+ // if (DEBUG) {
+ // System.out.println(" cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")" + " arc.output=" + arc.output + " output=" + output);
+ // }
+ if (cmp != 0) {
+ break;
+ }
+ arc = arcs[1+targetUpto];
+ //if (arc.label != (target.bytes[target.offset + targetUpto] & 0xFF)) {
+ //System.out.println("FAIL: arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF));
+ //}
+ assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
+ if (arc.output != NO_OUTPUT) {
+ output = fstOutputs.add(output, arc.output);
+ }
+ if (arc.isFinal()) {
+ lastFrame = stack[1+lastFrame.ord];
+ }
+ targetUpto++;
+ }
+
+ if (cmp == 0) {
+ final int targetUptoMid = targetUpto;
+
+ // Second compare the rest of the term, but
+ // don't save arc/output/frame; we only do this
+ // to find out if the target term is before,
+ // equal or after the current term
+ final int targetLimit2 = Math.min(target.length, term.length);
+ while (targetUpto < targetLimit2) {
+ cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+ // if (DEBUG) {
+ // System.out.println(" cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")");
+ // }
+ if (cmp != 0) {
+ break;
+ }
+ targetUpto++;
+ }
+
+ if (cmp == 0) {
+ cmp = term.length - target.length;
+ }
+ targetUpto = targetUptoMid;
+ }
+
+ if (cmp < 0) {
+ // Common case: target term is after current
+ // term, ie, app is seeking multiple terms
+ // in sorted order
+ // if (DEBUG) {
+ // System.out.println(" target is after current (shares prefixLen=" + targetUpto + "); frame.ord=" + lastFrame.ord);
+ // }
+ currentFrame = lastFrame;
+
+ } else if (cmp > 0) {
+ // Uncommon case: target term
+ // is before current term; this means we can
+ // keep the currentFrame but we must rewind it
+ // (so we scan from the start)
+ targetBeforeCurrentLength = 0;
+ // if (DEBUG) {
+ // System.out.println(" target is before current (shares prefixLen=" + targetUpto + "); rewind frame ord=" + lastFrame.ord);
+ // }
+ currentFrame = lastFrame;
+ currentFrame.rewind();
+ } else {
+ // Target is exactly the same as current term
+ assert term.length == target.length;
+ if (termExists) {
+ // if (DEBUG) {
+ // System.out.println(" target is same as current; return true");
+ // }
+ return true;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" target is same as current but term doesn't exist");
+ // }
+ }
+ //validIndexPrefix = currentFrame.depth;
+ //term.length = target.length;
+ //return termExists;
+ }
+
+ } else {
+
+ targetBeforeCurrentLength = -1;
+ arc = index.getFirstArc(arcs[0]);
+
+ // Empty string prefix must have an output (block) in the index!
+ assert arc.isFinal();
+ assert arc.output != null;
+
+ // if (DEBUG) {
+ // System.out.println(" no seek state; push root frame");
+ // }
+
+ output = arc.output;
+
+ currentFrame = staticFrame;
+
+ //term.length = 0;
+ targetUpto = 0;
+ currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), 0);
+ }
+
+ // if (DEBUG) {
+ // System.out.println(" start index loop targetUpto=" + targetUpto + " output=" + output + " currentFrame.ord=" + currentFrame.ord + " targetBeforeCurrentLength=" + targetBeforeCurrentLength);
+ // }
+
+ while (targetUpto < target.length) {
+
+ final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF;
+
+ final FST.Arc nextArc = index.findTargetArc(targetLabel, arc, getArc(1+targetUpto));
+
+ if (nextArc == null) {
+
+ // Index is exhausted
+ // if (DEBUG) {
+ // System.out.println(" index: index exhausted label=" + ((char) targetLabel) + " " + toHex(targetLabel));
+ // }
+
+ validIndexPrefix = currentFrame.prefix;
+ //validIndexPrefix = targetUpto;
+
+ currentFrame.scanToFloorFrame(target);
+
+ if (!currentFrame.hasTerms) {
+ termExists = false;
+ term.bytes[targetUpto] = (byte) targetLabel;
+ term.length = 1+targetUpto;
+ // if (DEBUG) {
+ // System.out.println(" FAST NOT_FOUND term=" + brToString(term));
+ // }
+ return false;
+ }
+
+ currentFrame.loadBlock();
+
+ final SeekStatus result = currentFrame.scanToTerm(target, true);
+ if (result == SeekStatus.FOUND) {
+ // if (DEBUG) {
+ // System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
+ // }
+ return true;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" got " + result + "; return NOT_FOUND term=" + brToString(term));
+ // }
+ return false;
+ }
+ } else {
+ // Follow this arc
+ arc = nextArc;
+ term.bytes[targetUpto] = (byte) targetLabel;
+ // Aggregate output as we go:
+ assert arc.output != null;
+ if (arc.output != NO_OUTPUT) {
+ output = fstOutputs.add(output, arc.output);
+ }
+
+ // if (DEBUG) {
+ // System.out.println(" index: follow label=" + toHex(target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
+ // }
+ targetUpto++;
+
+ if (arc.isFinal()) {
+ //if (DEBUG) System.out.println(" arc is final!");
+ currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), targetUpto);
+ //if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
+ }
+ }
+ }
+
+ //validIndexPrefix = targetUpto;
+ validIndexPrefix = currentFrame.prefix;
+
+ currentFrame.scanToFloorFrame(target);
+
+ // Target term is entirely contained in the index:
+ if (!currentFrame.hasTerms) {
+ termExists = false;
+ term.length = targetUpto;
+ // if (DEBUG) {
+ // System.out.println(" FAST NOT_FOUND term=" + brToString(term));
+ // }
+ return false;
+ }
+
+ currentFrame.loadBlock();
+
+ final SeekStatus result = currentFrame.scanToTerm(target, true);
+ if (result == SeekStatus.FOUND) {
+ // if (DEBUG) {
+ // System.out.println(" return FOUND term=" + term.utf8ToString() + " " + term);
+ // }
+ return true;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" got result " + result + "; return NOT_FOUND term=" + term.utf8ToString());
+ // }
+
+ return false;
+ }
+ }
+
+ @Override
+ public SeekStatus seekCeil(final BytesRef target, final boolean useCache) throws IOException {
+ if (index == null) {
+ throw new IllegalStateException("terms index was not loaded");
+ }
+
+ if (term.bytes.length <= target.length) {
+ term.bytes = ArrayUtil.grow(term.bytes, 1+target.length);
+ }
+
+ assert clearEOF();
+
+ //if (DEBUG) {
+ //System.out.println("\nBTTR.seekCeil seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + brToString(term) + " (exists?=" + termExists + ") validIndexPrefix= " + validIndexPrefix);
+ //printSeekState();
+ //}
+
+ FST.Arc arc;
+ int targetUpto;
+ BytesRef output;
+
+ targetBeforeCurrentLength = currentFrame.ord;
+
+ if (currentFrame != staticFrame) {
+
+ // We are already seek'd; find the common
+ // prefix of new seek term vs current term and
+ // re-use the corresponding seek state. For
+ // example, if app first seeks to foobar, then
+ // seeks to foobaz, we can re-use the seek state
+ // for the first 5 bytes.
+
+ //if (DEBUG) {
+ //System.out.println(" re-use current seek state validIndexPrefix=" + validIndexPrefix);
+ //}
+
+ arc = arcs[0];
+ assert arc.isFinal();
+ output = arc.output;
+ targetUpto = 0;
+
+ Frame lastFrame = stack[0];
+ assert validIndexPrefix <= term.length;
+
+ final int targetLimit = Math.min(target.length, validIndexPrefix);
+
+ int cmp = 0;
+
+ // TOOD: we should write our vLong backwards (MSB
+ // first) to get better sharing from the FST
+
+ // First compare up to valid seek frames:
+ while (targetUpto < targetLimit) {
+ cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+ //if (DEBUG) {
+ //System.out.println(" cycle targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")" + " arc.output=" + arc.output + " output=" + output);
+ //}
+ if (cmp != 0) {
+ break;
+ }
+ arc = arcs[1+targetUpto];
+ assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
+ // TOOD: we could save the outputs in local
+ // byte[][] instead of making new objs ever
+ // seek; but, often the FST doesn't have any
+ // shared bytes (but this could change if we
+ // reverse vLong byte order)
+ if (arc.output != NO_OUTPUT) {
+ output = fstOutputs.add(output, arc.output);
+ }
+ if (arc.isFinal()) {
+ lastFrame = stack[1+lastFrame.ord];
+ }
+ targetUpto++;
+ }
+
+
+ if (cmp == 0) {
+ final int targetUptoMid = targetUpto;
+ // Second compare the rest of the term, but
+ // don't save arc/output/frame:
+ final int targetLimit2 = Math.min(target.length, term.length);
+ while (targetUpto < targetLimit2) {
+ cmp = (term.bytes[targetUpto]&0xFF) - (target.bytes[target.offset + targetUpto]&0xFF);
+ //if (DEBUG) {
+ //System.out.println(" cycle2 targetUpto=" + targetUpto + " (vs limit=" + targetLimit + ") cmp=" + cmp + " (targetLabel=" + (char) (target.bytes[target.offset + targetUpto]) + " vs termLabel=" + (char) (term.bytes[targetUpto]) + ")");
+ //}
+ if (cmp != 0) {
+ break;
+ }
+ targetUpto++;
+ }
+
+ if (cmp == 0) {
+ cmp = term.length - target.length;
+ }
+ targetUpto = targetUptoMid;
+ }
+
+ if (cmp < 0) {
+ // Common case: target term is after current
+ // term, ie, app is seeking multiple terms
+ // in sorted order
+ //if (DEBUG) {
+ //System.out.println(" target is after current (shares prefixLen=" + targetUpto + "); clear frame.scanned ord=" + lastFrame.ord);
+ //}
+ currentFrame = lastFrame;
+
+ } else if (cmp > 0) {
+ // Uncommon case: target term
+ // is before current term; this means we can
+ // keep the currentFrame but we must rewind it
+ // (so we scan from the start)
+ targetBeforeCurrentLength = 0;
+ //if (DEBUG) {
+ //System.out.println(" target is before current (shares prefixLen=" + targetUpto + "); rewind frame ord=" + lastFrame.ord);
+ //}
+ currentFrame = lastFrame;
+ currentFrame.rewind();
+ } else {
+ // Target is exactly the same as current term
+ assert term.length == target.length;
+ if (termExists) {
+ //if (DEBUG) {
+ //System.out.println(" target is same as current; return FOUND");
+ //}
+ return SeekStatus.FOUND;
+ } else {
+ //if (DEBUG) {
+ //System.out.println(" target is same as current but term doesn't exist");
+ //}
+ }
+ }
+
+ } else {
+
+ targetBeforeCurrentLength = -1;
+ arc = index.getFirstArc(arcs[0]);
+
+ // Empty string prefix must have an output (block) in the index!
+ assert arc.isFinal();
+ assert arc.output != null;
+
+ //if (DEBUG) {
+ //System.out.println(" no seek state; push root frame");
+ //}
+
+ output = arc.output;
+
+ currentFrame = staticFrame;
+
+ //term.length = 0;
+ targetUpto = 0;
+ currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), 0);
+ }
+
+ //if (DEBUG) {
+ //System.out.println(" start index loop targetUpto=" + targetUpto + " output=" + output + " currentFrame.ord+1=" + currentFrame.ord + " targetBeforeCurrentLength=" + targetBeforeCurrentLength);
+ //}
+
+ while (targetUpto < target.length) {
+
+ final int targetLabel = target.bytes[target.offset + targetUpto] & 0xFF;
+
+ final FST.Arc nextArc = index.findTargetArc(targetLabel, arc, getArc(1+targetUpto));
+
+ if (nextArc == null) {
+
+ // Index is exhausted
+ // if (DEBUG) {
+ // System.out.println(" index: index exhausted label=" + ((char) targetLabel) + " " + toHex(targetLabel));
+ // }
+
+ validIndexPrefix = currentFrame.prefix;
+ //validIndexPrefix = targetUpto;
+
+ currentFrame.scanToFloorFrame(target);
+
+ currentFrame.loadBlock();
+
+ final SeekStatus result = currentFrame.scanToTerm(target, false);
+ if (result == SeekStatus.END) {
+ term.copy(target);
+ termExists = false;
+
+ if (next() != null) {
+ //if (DEBUG) {
+ //System.out.println(" return NOT_FOUND term=" + brToString(term) + " " + term);
+ //}
+ return SeekStatus.NOT_FOUND;
+ } else {
+ //if (DEBUG) {
+ //System.out.println(" return END");
+ //}
+ return SeekStatus.END;
+ }
+ } else {
+ //if (DEBUG) {
+ //System.out.println(" return " + result + " term=" + brToString(term) + " " + term);
+ //}
+ return result;
+ }
+ } else {
+ // Follow this arc
+ term.bytes[targetUpto] = (byte) targetLabel;
+ arc = nextArc;
+ // Aggregate output as we go:
+ assert arc.output != null;
+ if (arc.output != NO_OUTPUT) {
+ output = fstOutputs.add(output, arc.output);
+ }
+
+ //if (DEBUG) {
+ //System.out.println(" index: follow label=" + toHex(target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
+ //}
+ targetUpto++;
+
+ if (arc.isFinal()) {
+ //if (DEBUG) System.out.println(" arc is final!");
+ currentFrame = pushFrame(arc, fstOutputs.add(output, arc.nextFinalOutput), targetUpto);
+ //if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
+ }
+ }
+ }
+
+ //validIndexPrefix = targetUpto;
+ validIndexPrefix = currentFrame.prefix;
+
+ currentFrame.scanToFloorFrame(target);
+
+ currentFrame.loadBlock();
+
+ final SeekStatus result = currentFrame.scanToTerm(target, false);
+
+ if (result == SeekStatus.END) {
+ term.copy(target);
+ termExists = false;
+ if (next() != null) {
+ //if (DEBUG) {
+ //System.out.println(" return NOT_FOUND term=" + term.utf8ToString() + " " + term);
+ //}
+ return SeekStatus.NOT_FOUND;
+ } else {
+ //if (DEBUG) {
+ //System.out.println(" return END");
+ //}
+ return SeekStatus.END;
+ }
+ } else {
+ return result;
+ }
+ }
+
+ private void printSeekState() throws IOException {
+ if (currentFrame == staticFrame) {
+ System.out.println(" no prior seek");
+ } else {
+ System.out.println(" prior seek state:");
+ int ord = 0;
+ boolean isSeekFrame = true;
+ while(true) {
+ Frame f = getFrame(ord);
+ assert f != null;
+ final BytesRef prefix = new BytesRef(term.bytes, 0, f.prefix);
+ if (f.nextEnt == -1) {
+ System.out.println(" frame " + (isSeekFrame ? "(seek)" : "(next)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp< 0 || fp != fpOrig) {
+ if (DEBUG) {
+ System.out.println(" rewind frame ord=" + ord + " fpOrig=" + fpOrig + " fp=" + fp + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " nextEnt=" + nextEnt + " prefixLen=" + prefix);
+ }
+ if (fp != fpOrig) {
+ fp = fpOrig;
+ nextEnt = -1;
+ } else {
+ nextEnt = 0;
+ }
+ hasTerms = hasTermsOrig;
+ if (isFloor) {
+ floorDataReader.rewind();
+ numFollowFloorBlocks = floorDataReader.readVInt();
+ nextFloorLabel = floorDataReader.readByte() & 0xff;
+ }
+ assert suffixBytes != null;
+ suffixesReader.rewind();
+ assert statBytes != null;
+ statsReader.rewind();
+ metaDataUpto = 0;
+ state.termBlockOrd = 0;
+ // TODO: skip this if !hasTerms? Then postings
+ // impl wouldn't have to write useless 0 byte
+ postingsReader.resetTermsBlock(fieldInfo, state);
+ lastSubFP = -1;
+ } else if (DEBUG) {
+ System.out.println(" skip rewind fp=" + fp + " fpOrig=" + fpOrig + " nextEnt=" + nextEnt + " ord=" + ord);
+ }
+ */
+ }
+
+ public boolean next() {
+ return isLeafBlock ? nextLeaf() : nextNonLeaf();
+ }
+
+ // Decodes next entry; returns true if it's a sub-block
+ public boolean nextLeaf() {
+ //if (DEBUG) System.out.println(" frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+ assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+ nextEnt++;
+ suffix = suffixesReader.readVInt();
+ startBytePos = suffixesReader.getPosition();
+ term.length = prefix + suffix;
+ if (term.bytes.length < term.length) {
+ term.grow(term.length);
+ }
+ suffixesReader.readBytes(term.bytes, prefix, suffix);
+ // A normal term
+ termExists = true;
+ return false;
+ }
+
+ public boolean nextNonLeaf() {
+ //if (DEBUG) System.out.println(" frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+ assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+ nextEnt++;
+ final int code = suffixesReader.readVInt();
+ suffix = code >>> 1;
+ startBytePos = suffixesReader.getPosition();
+ term.length = prefix + suffix;
+ if (term.bytes.length < term.length) {
+ term.grow(term.length);
+ }
+ suffixesReader.readBytes(term.bytes, prefix, suffix);
+ if ((code & 1) == 0) {
+ // A normal term
+ termExists = true;
+ state.termBlockOrd++;
+ return false;
+ } else {
+ // A sub-block; make sub-FP absolute:
+ termExists = false;
+ lastSubFP = fp - suffixesReader.readVLong();
+ //if (DEBUG) {
+ //System.out.println(" lastSubFP=" + lastSubFP);
+ //}
+ return true;
+ }
+ }
+
+ // TODO: make this array'd so we can do bin search?
+ // likely not worth it? need to measure how many
+ // floor blocks we "typically" get
+ public void scanToFloorFrame(BytesRef target) {
+
+ if (!isFloor || target.length <= prefix) {
+ // if (DEBUG) {
+ // System.out.println(" scanToFloorFrame skip: isFloor=" + isFloor + " target.length=" + target.length + " vs prefix=" + prefix);
+ // }
+ return;
+ }
+
+ final int targetLabel = target.bytes[target.offset + prefix] & 0xFF;
+
+ // if (DEBUG) {
+ // System.out.println(" scanToFloorFrame fpOrig=" + fpOrig + " targetLabel=" + toHex(targetLabel) + " vs nextFloorLabel=" + toHex(nextFloorLabel) + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+ // }
+
+ if (targetLabel < nextFloorLabel) {
+ // if (DEBUG) {
+ // System.out.println(" already on correct block");
+ // }
+ return;
+ }
+
+ assert numFollowFloorBlocks != 0;
+
+ long newFP = fpOrig;
+ while (true) {
+ final long code = floorDataReader.readVLong();
+ newFP = fpOrig + (code >>> 1);
+ hasTerms = (code & 1) != 0;
+ // if (DEBUG) {
+ // System.out.println(" label=" + toHex(nextFloorLabel) + " fp=" + newFP + " hasTerms?=" + hasTerms + " numFollowFloor=" + numFollowFloorBlocks);
+ // }
+
+ isLastInFloor = numFollowFloorBlocks == 1;
+ numFollowFloorBlocks--;
+
+ if (isLastInFloor) {
+ nextFloorLabel = 256;
+ // if (DEBUG) {
+ // System.out.println(" stop! last block nextFloorLabel=" + toHex(nextFloorLabel));
+ // }
+ break;
+ } else {
+ nextFloorLabel = floorDataReader.readByte() & 0xff;
+ if (targetLabel < nextFloorLabel) {
+ // if (DEBUG) {
+ // System.out.println(" stop! nextFloorLabel=" + toHex(nextFloorLabel));
+ // }
+ break;
+ }
+ }
+ }
+
+ if (newFP != fp) {
+ // Force re-load of the block:
+ // if (DEBUG) {
+ // System.out.println(" force switch to fp=" + newFP + " oldFP=" + fp);
+ // }
+ nextEnt = -1;
+ fp = newFP;
+ } else {
+ // if (DEBUG) {
+ // System.out.println(" stay on same fp=" + newFP);
+ // }
+ }
+ }
+
+ public void decodeMetaData() throws IOException {
+
+ //if (DEBUG) System.out.println("\nBTTR.decodeMetadata seg=" + segment + " mdUpto=" + metaDataUpto + " vs termBlockOrd=" + state.termBlockOrd);
+
+ // lazily catch up on metadata decode:
+ final int limit = getTermBlockOrd();
+ assert limit > 0;
+
+ // We must set/incr state.termCount because
+ // postings impl can look at this
+ state.termBlockOrd = metaDataUpto;
+
+ // TODO: better API would be "jump straight to term=N"???
+ while (metaDataUpto < limit) {
+
+ // TODO: we could make "tiers" of metadata, ie,
+ // decode docFreq/totalTF but don't decode postings
+ // metadata; this way caller could get
+ // docFreq/totalTF w/o paying decode cost for
+ // postings
+
+ // TODO: if docFreq were bulk decoded we could
+ // just skipN here:
+ state.docFreq = statsReader.readVInt();
+ //if (DEBUG) System.out.println(" dF=" + state.docFreq);
+ if (fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) {
+ state.totalTermFreq = state.docFreq + statsReader.readVLong();
+ //if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq);
+ }
+
+ postingsReader.nextTerm(fieldInfo, state);
+ metaDataUpto++;
+ state.termBlockOrd++;
+ }
+ }
+
+ // Used only by assert
+ private boolean prefixMatches(BytesRef target) {
+ for(int bytePos=0;bytePos fields = new ArrayList();
+ // private final String segment;
+
+ /** Create a new writer. The number of items (terms or
+ * sub-blocks) per block will aim to be between
+ * minItemsPerBlock and maxItemsPerBlock, though in some
+ * cases the blocks may be smaller than the min. */
+ public BlockTreeTermsWriter(
+ SegmentWriteState state,
+ PostingsWriterBase postingsWriter,
+ int minItemsInBlock,
+ int maxItemsInBlock)
+ throws IOException
+ {
+ if (minItemsInBlock <= 1) {
+ throw new IllegalArgumentException("minItemsInBlock must be >= 2; got " + minItemsInBlock);
+ }
+ if (maxItemsInBlock <= 0) {
+ throw new IllegalArgumentException("maxItemsInBlock must be >= 1; got " + maxItemsInBlock);
+ }
+ if (minItemsInBlock > maxItemsInBlock) {
+ throw new IllegalArgumentException("maxItemsInBlock must be >= minItemsInBlock; got maxItemsInBlock=" + maxItemsInBlock + " minItemsInBlock=" + minItemsInBlock);
+ }
+ if (2*(minItemsInBlock-1) > maxItemsInBlock) {
+ throw new IllegalArgumentException("maxItemsInBlock must be at least 2*(minItemsInBlock-1); got maxItemsInBlock=" + maxItemsInBlock + " minItemsInBlock=" + minItemsInBlock);
+ }
+
+ final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
+ out = state.directory.createOutput(termsFileName, state.context);
+ boolean success = false;
+ IndexOutput indexOut = null;
+ try {
+ fieldInfos = state.fieldInfos;
+ this.minItemsInBlock = minItemsInBlock;
+ this.maxItemsInBlock = maxItemsInBlock;
+ writeHeader(out);
+
+ //DEBUG = state.segmentName.equals("_4a");
+
+ final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
+ indexOut = state.directory.createOutput(termsIndexFileName, state.context);
+ writeIndexHeader(indexOut);
+
+ currentField = null;
+ this.postingsWriter = postingsWriter;
+ // segment = state.segmentName;
+
+ // System.out.println("BTW.init seg=" + state.segmentName);
+
+ postingsWriter.start(out); // have consumer write its format/header
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeSafely(true, out, indexOut);
+ }
+ }
+ this.indexOut = indexOut;
+ }
+
+ protected void writeHeader(IndexOutput out) throws IOException {
+ CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
+ out.writeLong(0); // leave space for end index pointer
+ }
+
+ protected void writeIndexHeader(IndexOutput out) throws IOException {
+ CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
+ out.writeLong(0); // leave space for end index pointer
+ }
+
+ protected void writeTrailer(long dirStart) throws IOException {
+ out.seek(CodecUtil.headerLength(CODEC_NAME));
+ out.writeLong(dirStart);
+ }
+
+ protected void writeIndexTrailer(long dirStart) throws IOException {
+ indexOut.seek(CodecUtil.headerLength(CODEC_NAME));
+ indexOut.writeLong(dirStart);
+ }
+
+ @Override
+ public TermsConsumer addField(FieldInfo field) throws IOException {
+ //DEBUG = field.name.equals("id");
+ //if (DEBUG) System.out.println("\nBTTW.addField seg=" + segment + " field=" + field.name);
+ assert currentField == null || currentField.name.compareTo(field.name) < 0;
+ currentField = field;
+ final TermsWriter terms = new TermsWriter(field);
+ fields.add(terms);
+ return terms;
+ }
+
+ static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) {
+ assert fp < (1L << 62);
+ return (fp << 2) | (hasTerms ? OUTPUT_FLAG_HAS_TERMS : 0) | (isFloor ? OUTPUT_FLAG_IS_FLOOR : 0);
+ }
+
+ private static class PendingEntry {
+ public final boolean isTerm;
+
+ protected PendingEntry(boolean isTerm) {
+ this.isTerm = isTerm;
+ }
+ }
+
+ private static final class PendingTerm extends PendingEntry {
+ public final BytesRef term;
+ public final TermStats stats;
+
+ public PendingTerm(BytesRef term, TermStats stats) {
+ super(true);
+ this.term = term;
+ this.stats = stats;
+ }
+
+ @Override
+ public String toString() {
+ return term.utf8ToString();
+ }
+ }
+
+ private static final class PendingBlock extends PendingEntry {
+ public final BytesRef prefix;
+ public final long fp;
+ public FST index;
+ public List> subIndices;
+ public final boolean hasTerms;
+ public final boolean isFloor;
+ public final int floorLeadByte;
+
+ public PendingBlock(BytesRef prefix, long fp, boolean hasTerms, boolean isFloor, int floorLeadByte, List> subIndices) {
+ super(false);
+ this.prefix = prefix;
+ this.fp = fp;
+ this.hasTerms = hasTerms;
+ this.isFloor = isFloor;
+ this.floorLeadByte = floorLeadByte;
+ this.subIndices = subIndices;
+ }
+
+ @Override
+ public String toString() {
+ return "BLOCK: " + prefix.utf8ToString();
+ }
+
+ public void compileIndex(List floorBlocks, RAMOutputStream scratchBytes) throws IOException {
+
+ assert (isFloor && floorBlocks != null && floorBlocks.size() != 0) || (!isFloor && floorBlocks == null): "isFloor=" + isFloor + " floorBlocks=" + floorBlocks;
+
+ assert scratchBytes.getFilePointer() == 0;
+
+ // TODO: try writing the leading vLong in MSB order
+ // (opposite of what Lucene does today), for better
+ // outputs sharing in the FST
+ scratchBytes.writeVLong(encodeOutput(fp, hasTerms, isFloor));
+ if (isFloor) {
+ scratchBytes.writeVInt(floorBlocks.size());
+ for (PendingBlock sub : floorBlocks) {
+ assert sub.floorLeadByte != -1;
+ //if (DEBUG) {
+ // System.out.println(" write floorLeadByte=" + Integer.toHexString(sub.floorLeadByte&0xff));
+ //}
+ scratchBytes.writeByte((byte) sub.floorLeadByte);
+ assert sub.fp > fp;
+ scratchBytes.writeVLong((sub.fp - fp) << 1 | (sub.hasTerms ? 1 : 0));
+ }
+ }
+
+ final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
+ final Builder indexBuilder = new Builder(FST.INPUT_TYPE.BYTE1,
+ 0, 0, true, false, Integer.MAX_VALUE,
+ outputs, null);
+ //if (DEBUG) {
+ // System.out.println(" compile index for prefix=" + prefix);
+ //}
+ //indexBuilder.DEBUG = false;
+ final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()];
+ assert bytes.length > 0;
+ scratchBytes.writeTo(bytes, 0);
+ indexBuilder.add(prefix, new BytesRef(bytes, 0, bytes.length));
+ scratchBytes.reset();
+
+ // Copy over index for all sub-blocks
+
+ if (subIndices != null) {
+ for(FST subIndex : subIndices) {
+ append(indexBuilder, subIndex);
+ }
+ }
+
+ if (floorBlocks != null) {
+ for (PendingBlock sub : floorBlocks) {
+ if (sub.subIndices != null) {
+ for(FST subIndex : sub.subIndices) {
+ append(indexBuilder, subIndex);
+ }
+ }
+ sub.subIndices = null;
+ }
+ }
+
+ index = indexBuilder.finish();
+ subIndices = null;
+
+ /*
+ Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"));
+ Util.toDot(index, w, false, false);
+ System.out.println("SAVED to out.dot");
+ w.close();
+ */
+ }
+
+ // TODO: maybe we could add bulk-add method to
+ // Builder? Takes FST and unions it w/ current
+ // FST.
+ private void append(Builder builder, FST subIndex) throws IOException {
+ final BytesRefFSTEnum subIndexEnum = new BytesRefFSTEnum(subIndex);
+ BytesRefFSTEnum.InputOutput indexEnt;
+ while((indexEnt = subIndexEnum.next()) != null) {
+ //if (DEBUG) {
+ // System.out.println(" add sub=" + indexEnt.input + " " + indexEnt.input + " output=" + indexEnt.output);
+ //}
+ builder.add(indexEnt.input, indexEnt.output);
+ }
+ }
+ }
+
+ final RAMOutputStream scratchBytes = new RAMOutputStream();
+
+ class TermsWriter extends TermsConsumer {
+ private final FieldInfo fieldInfo;
+ private long numTerms;
+ long sumTotalTermFreq;
+ long sumDocFreq;
+ long indexStartFP;
+
+ // Used only to partition terms into the block tree; we
+ // don't pull an FST from this builder:
+ private final NoOutputs noOutputs;
+ private final Builder