Index: lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
--- lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java Mon Aug 01 12:07:14 2011 -0400
+++ lucene/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java Mon Aug 01 12:09:21 2011 -0400
@@ -208,7 +208,7 @@
AttributeSource atts = new AttributeSource();
MaxNonCompetitiveBoostAttribute maxBoostAtt =
atts.addAttribute(MaxNonCompetitiveBoostAttribute.class);
- FuzzyTermsEnum fe = new FuzzyTermsEnum(MultiFields.getTerms(reader, startTerm.field()).iterator(), atts, startTerm, f.minSimilarity, f.prefixLength);
+ FuzzyTermsEnum fe = new FuzzyTermsEnum(MultiFields.getTerms(reader, startTerm.field()), atts, startTerm, f.minSimilarity, f.prefixLength);
//store the df so all variants use same idf
int df = reader.docFreq(startTerm);
int numVariants=0;
Index: lucene/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
--- /dev/null Thu Jan 01 00:00:00 1970 +0000
+++ lucene/src/java/org/apache/lucene/index/AutomatonTermsEnum.java Mon Aug 01 12:09:21 2011 -0400
@@ -0,0 +1,324 @@
+package org.apache.lucene.index;
+
+/**
+ * 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.IOException;
+import java.util.Comparator;
+
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.FilteredTermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.ByteRunAutomaton;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.SpecialOperations;
+import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.automaton.UTF32ToUTF8;
+
+/**
+ * A FilteredTermsEnum that enumerates terms based upon what is accepted by a
+ * DFA.
+ *
+ * The algorithm is such:
+ *
+ *
As long as matches are successful, keep reading sequentially.
+ *
When a match fails, skip to the next string in lexicographic order that
+ * does not enter a reject state.
+ *
+ *
+ * The algorithm does not attempt to actually skip to the next string that is
+ * completely accepted. This is not possible when the language accepted by the
+ * FSM is not finite (i.e. * operator).
+ *
+ * @lucene.experimental
+ */
+public class AutomatonTermsEnum extends FilteredTermsEnum {
+ // a tableized array-based form of the DFA
+ private final ByteRunAutomaton runAutomaton;
+ // common suffix of the automaton
+ private final BytesRef commonSuffixRef;
+ // true if the automaton accepts a finite language
+ private final boolean finite;
+ // array of sorted transitions for each state, indexed by state number
+ private final Transition[][] allTransitions;
+ // for path tracking: each long records gen when we last
+ // visited the state; we use gens to avoid having to clear
+ private final long[] visited;
+ private long curGen;
+ // the reference used for seeking forwards through the term dictionary
+ private final BytesRef seekBytesRef = new BytesRef(10);
+ // true if we are enumerating an infinite portion of the DFA.
+ // in this case it is faster to drive the query based on the terms dictionary.
+ // when this is true, linearUpperBound indicate the end of range
+ // of terms where we should simply do sequential reads instead.
+ private boolean linear = false;
+ private final BytesRef linearUpperBound = new BytesRef(10);
+ private final Comparator termComp;
+
+ /**
+ * Construct an enumerator based upon an automaton, enumerating the specified
+ * field, working on a supplied TermsEnum
+ *
+ * @lucene.experimental
+ *
+ * @param compiled CompiledAutomaton
+ */
+ public AutomatonTermsEnum(TermsEnum tenum, CompiledAutomaton compiled) throws IOException {
+ super(tenum);
+ this.finite = compiled.finite;
+ this.runAutomaton = compiled.runAutomaton;
+ this.commonSuffixRef = compiled.commonSuffixRef;
+ this.allTransitions = compiled.sortedTransitions;
+
+ // used for path tracking, where each bit is a numbered state.
+ visited = new long[runAutomaton.getSize()];
+
+ termComp = getComparator();
+ }
+
+ /**
+ * Returns true if the term matches the automaton. Also stashes away the term
+ * to assist with smart enumeration.
+ */
+ @Override
+ protected AcceptStatus accept(final BytesRef term) {
+ if (commonSuffixRef == null || term.endsWith(commonSuffixRef)) {
+ if (runAutomaton.run(term.bytes, term.offset, term.length))
+ return linear ? AcceptStatus.YES : AcceptStatus.YES_AND_SEEK;
+ else
+ return (linear && termComp.compare(term, linearUpperBound) < 0) ?
+ AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
+ } else {
+ return (linear && termComp.compare(term, linearUpperBound) < 0) ?
+ AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
+ }
+ }
+
+ @Override
+ protected BytesRef nextSeekTerm(final BytesRef term) throws IOException {
+ //System.out.println("ATE.nextSeekTerm term=" + term);
+ if (term == null) {
+ assert seekBytesRef.length == 0;
+ // return the empty term, as its valid
+ if (runAutomaton.isAccept(runAutomaton.getInitialState())) {
+ return seekBytesRef;
+ }
+ } else {
+ seekBytesRef.copy(term);
+ }
+
+ // seek to the next possible string;
+ if (nextString()) {
+ return seekBytesRef; // reposition
+ } else {
+ return null; // no more possible strings can match
+ }
+ }
+
+ /**
+ * Sets the enum to operate in linear fashion, as we have found
+ * a looping transition at position: we set an upper bound and
+ * act like a TermRangeQuery for this portion of the term space.
+ */
+ private void setLinear(int position) {
+ assert linear == false;
+
+ int state = runAutomaton.getInitialState();
+ int maxInterval = 0xff;
+ for (int i = 0; i < position; i++) {
+ state = runAutomaton.step(state, seekBytesRef.bytes[i] & 0xff);
+ assert state >= 0: "state=" + state;
+ }
+ for (int i = 0; i < allTransitions[state].length; i++) {
+ Transition t = allTransitions[state][i];
+ if (t.getMin() <= (seekBytesRef.bytes[position] & 0xff) &&
+ (seekBytesRef.bytes[position] & 0xff) <= t.getMax()) {
+ maxInterval = t.getMax();
+ break;
+ }
+ }
+ // 0xff terms don't get the optimization... not worth the trouble.
+ if (maxInterval != 0xff)
+ maxInterval++;
+ int length = position + 1; /* position + maxTransition */
+ if (linearUpperBound.bytes.length < length)
+ linearUpperBound.bytes = new byte[length];
+ System.arraycopy(seekBytesRef.bytes, 0, linearUpperBound.bytes, 0, position);
+ linearUpperBound.bytes[position] = (byte) maxInterval;
+ linearUpperBound.length = length;
+
+ linear = true;
+ }
+
+ private final IntsRef savedStates = new IntsRef(10);
+
+ /**
+ * Increments the byte buffer to the next String in binary order after s that will not put
+ * the machine into a reject state. If such a string does not exist, returns
+ * false.
+ *
+ * The correctness of this method depends upon the automaton being deterministic,
+ * and having no transitions to dead states.
+ *
+ * @return true if more possible solutions exist for the DFA
+ */
+ private boolean nextString() {
+ int state;
+ int pos = 0;
+ savedStates.grow(seekBytesRef.length+1);
+ final int[] states = savedStates.ints;
+ states[0] = runAutomaton.getInitialState();
+
+ while (true) {
+ curGen++;
+ linear = false;
+ // walk the automaton until a character is rejected.
+ for (state = states[pos]; pos < seekBytesRef.length; pos++) {
+ visited[state] = curGen;
+ int nextState = runAutomaton.step(state, seekBytesRef.bytes[pos] & 0xff);
+ if (nextState == -1)
+ break;
+ states[pos+1] = nextState;
+ // we found a loop, record it for faster enumeration
+ if (!finite && !linear && visited[nextState] == curGen) {
+ setLinear(pos);
+ }
+ state = nextState;
+ }
+
+ // take the useful portion, and the last non-reject state, and attempt to
+ // append characters that will match.
+ if (nextString(state, pos)) {
+ return true;
+ } else { /* no more solutions exist from this useful portion, backtrack */
+ if ((pos = backtrack(pos)) < 0) /* no more solutions at all */
+ return false;
+ final int newState = runAutomaton.step(states[pos], seekBytesRef.bytes[pos] & 0xff);
+ if (newState >= 0 && runAutomaton.isAccept(newState))
+ /* String is good to go as-is */
+ return true;
+ /* else advance further */
+ // TODO: paranoia? if we backtrack thru an infinite DFA, the loop detection is important!
+ // for now, restart from scratch for all infinite DFAs
+ if (!finite) pos = 0;
+ }
+ }
+ }
+
+ /**
+ * Returns the next String in lexicographic order that will not put
+ * the machine into a reject state.
+ *
+ * This method traverses the DFA from the given position in the String,
+ * starting at the given state.
+ *
+ * If this cannot satisfy the machine, returns false. This method will
+ * walk the minimal path, in lexicographic order, as long as possible.
+ *
+ * If this method returns false, then there might still be more solutions,
+ * it is necessary to backtrack to find out.
+ *
+ * @param state current non-reject state
+ * @param position useful portion of the string
+ * @return true if more possible solutions exist for the DFA from this
+ * position
+ */
+ private boolean nextString(int state, int position) {
+ /*
+ * the next lexicographic character must be greater than the existing
+ * character, if it exists.
+ */
+ int c = 0;
+ if (position < seekBytesRef.length) {
+ c = seekBytesRef.bytes[position] & 0xff;
+ // if the next byte is 0xff and is not part of the useful portion,
+ // then by definition it puts us in a reject state, and therefore this
+ // path is dead. there cannot be any higher transitions. backtrack.
+ if (c++ == 0xff)
+ return false;
+ }
+
+ seekBytesRef.length = position;
+ visited[state] = curGen;
+
+ Transition transitions[] = allTransitions[state];
+
+ // find the minimal path (lexicographic order) that is >= c
+
+ for (int i = 0; i < transitions.length; i++) {
+ Transition transition = transitions[i];
+ if (transition.getMax() >= c) {
+ int nextChar = Math.max(c, transition.getMin());
+ // append either the next sequential char, or the minimum transition
+ seekBytesRef.grow(seekBytesRef.length + 1);
+ seekBytesRef.length++;
+ seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) nextChar;
+ state = transition.getDest().getNumber();
+ /*
+ * as long as is possible, continue down the minimal path in
+ * lexicographic order. if a loop or accept state is encountered, stop.
+ */
+ while (visited[state] != curGen && !runAutomaton.isAccept(state)) {
+ visited[state] = curGen;
+ /*
+ * Note: we work with a DFA with no transitions to dead states.
+ * so the below is ok, if it is not an accept state,
+ * then there MUST be at least one transition.
+ */
+ transition = allTransitions[state][0];
+ state = transition.getDest().getNumber();
+
+ // append the minimum transition
+ seekBytesRef.grow(seekBytesRef.length + 1);
+ seekBytesRef.length++;
+ seekBytesRef.bytes[seekBytesRef.length - 1] = (byte) transition.getMin();
+
+ // we found a loop, record it for faster enumeration
+ if (!finite && !linear && visited[state] == curGen) {
+ setLinear(seekBytesRef.length-1);
+ }
+ }
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Attempts to backtrack thru the string after encountering a dead end
+ * at some given position. Returns false if no more possible strings
+ * can match.
+ *
+ * @param position current position in the input String
+ * @return position >=0 if more possible solutions exist for the DFA
+ */
+ private int backtrack(int position) {
+ while (position-- > 0) {
+ int nextChar = seekBytesRef.bytes[position] & 0xff;
+ // if a character is 0xff its a dead-end too,
+ // because there is no higher character in binary sort order.
+ if (nextChar++ != 0xff) {
+ seekBytesRef.bytes[position] = (byte) nextChar;
+ seekBytesRef.length = position+1;
+ return position;
+ }
+ }
+ return -1; /* all solutions exhausted */
+ }
+}
Index: lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java
--- lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java Mon Aug 01 12:07:14 2011 -0400
+++ lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java Mon Aug 01 12:09:21 2011 -0400
@@ -397,10 +397,12 @@
if (termsEnum.seekExact(term.bytes(), false)) {
DocsEnum docsEnum = termsEnum.docs(reader.getLiveDocs(), docs);
+ //System.out.println("BDS: got docsEnum=" + docsEnum);
if (docsEnum != null) {
while (true) {
final int docID = docsEnum.nextDoc();
+ //System.out.println("BDS: docID=" + docID);
if (docID == DocsEnum.NO_MORE_DOCS) {
break;
}
Index: lucene/src/java/org/apache/lucene/index/CheckIndex.java
--- lucene/src/java/org/apache/lucene/index/CheckIndex.java Mon Aug 01 12:07:14 2011 -0400
+++ lucene/src/java/org/apache/lucene/index/CheckIndex.java Mon Aug 01 12:09:21 2011 -0400
@@ -888,7 +888,9 @@
// Test seek to last term:
if (lastTerm != null) {
- if (terms.seekCeil(lastTerm) != TermsEnum.SeekStatus.FOUND) {
+ if (terms.seekCeil(lastTerm) != TermsEnum.SeekStatus.FOUND) {
+ // nocommit
+ System.out.println("CI: seek to last term " + lastTerm + " failed");
throw new RuntimeException("seek to last term " + lastTerm + " failed");
}
Index: lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
--- lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Mon Aug 01 12:07:14 2011 -0400
+++ lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Mon Aug 01 12:09:21 2011 -0400
@@ -26,6 +26,7 @@
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 FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
+ boolean success = false;
+
try {
TermsHash termsHash = null;
@@ -100,8 +103,9 @@
if (termsHash != null) {
termsHash.reset();
}
+ success = true;
} finally {
- consumer.close();
+ IOUtils.closeSafely(!success, consumer);
}
}
Index: lucene/src/java/org/apache/lucene/index/SegmentMerger.java
--- lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon Aug 01 12:07:14 2011 -0400
+++ lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon Aug 01 12:09:21 2011 -0400
@@ -562,12 +562,14 @@
}
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);
}
}
Index: lucene/src/java/org/apache/lucene/index/Terms.java
--- lucene/src/java/org/apache/lucene/index/Terms.java Mon Aug 01 12:07:14 2011 -0400
+++ lucene/src/java/org/apache/lucene/index/Terms.java Mon Aug 01 12:09:21 2011 -0400
@@ -19,9 +19,11 @@
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,6 +39,38 @@
/** 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
. */
+ // nocommit need direct tests of this:
+ // nocommit not great that we pass startTerm... better to
+ // fully support .seekXXX in the returned enum?
+ public TermsEnum intersect(CompiledAutomaton compiled, final BytesRef startTerm) throws IOException {
+ //System.out.println("Terms.intersect compiled=" + compiled + " startTerm=" + startTerm);
+ //new Throwable().printStackTrace(System.out);
+ 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
Index: lucene/src/java/org/apache/lucene/index/TermsEnum.java
--- lucene/src/java/org/apache/lucene/index/TermsEnum.java Mon Aug 01 12:07:14 2011 -0400
+++ lucene/src/java/org/apache/lucene/index/TermsEnum.java Mon Aug 01 12:09:21 2011 -0400
@@ -42,6 +42,22 @@
* @lucene.experimental */
public abstract class TermsEnum {
+ // nocommit
+ public static int blockLoadCount;
+ public static int blockTermScanCount;
+ public static int blockSubScanCount;
+ public static int blockMDScanCount;
+
+ // nocommit
+ public final static String getStats() {
+ final String s = "blockLoad=" + blockLoadCount + " termScan=" + blockTermScanCount + " subScan=" + blockSubScanCount + " mdScan=" + blockMDScanCount;
+ blockLoadCount = 0;
+ blockTermScanCount = 0;
+ blockSubScanCount = 0;
+ blockMDScanCount = 0;
+ return s;
+ }
+
private AttributeSource atts = null;
/** Returns the related attributes. */
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
--- lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java Mon Aug 01 12:07:14 2011 -0400
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java Mon Aug 01 12:09:21 2011 -0400
@@ -45,6 +45,10 @@
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.util.DoubleBarrelLRUCache;
+// nocommit -- cut the postings interface over to the same
+// as blocktree? this way we don't need 2x of each codec's
+// postings impls?
+
/** Handles a terms dict, but decouples all details of
* doc/freqs/positions reading to an instance of {@link
* PostingsReaderBase}. This class is reusable for
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTreePostingsReaderBase.java
--- /dev/null Thu Jan 01 00:00:00 1970 +0000
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTreePostingsReaderBase.java Mon Aug 01 12:09:21 2011 -0400
@@ -0,0 +1,66 @@
+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.IOException;
+import java.io.Closeable;
+
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
+
+import org.apache.lucene.index.codecs.standard.StandardPostingsWriter; // javadocs
+
+/** BlockTreeTermsReader interacts with a single instance
+ * of this class to manage creation of {@link DocsEnum} and
+ * {@link DocsAndPositionsEnum} instances. It provides an
+ * IndexInput (termsIn) where this class may read any
+ * previously stored data that it had written in its
+ * corresponding {@link PostingsWriterBase} at indexing
+ * time.
+ * @lucene.experimental */
+
+public abstract class BlockTreePostingsReaderBase implements Closeable {
+
+ public abstract void init(IndexInput termsIn) throws IOException;
+
+ /** Return a newly created empty TermState */
+ public abstract BlockTreeTermState newTermState() throws IOException;
+
+ /** Actually decode metadata for next term */
+ public abstract void nextTerm(FieldInfo fieldInfo, BlockTreeTermState state) throws IOException;
+
+ /** Must fully consume state, since after this call that
+ * TermState may be reused. */
+ public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTreeTermState state, Bits skipDocs, DocsEnum reuse) throws IOException;
+
+ /** Must fully consume state, since after this call that
+ * TermState may be reused. */
+ public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTreeTermState state, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException;
+
+ public abstract void close() throws IOException;
+
+ /** Reads data for all terms in the next block; this
+ * method should merely load the byte[] blob but not
+ * decode, which is done in {@link #nextTerm}. */
+ public abstract void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTreeTermState termState) throws IOException;
+
+ public abstract void resetTermsBlock(FieldInfo fieldInfo, BlockTreeTermState termState) throws IOException;
+}
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTreePostingsWriterBase.java
--- /dev/null Thu Jan 01 00:00:00 1970 +0000
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTreePostingsWriterBase.java Mon Aug 01 12:09:21 2011 -0400
@@ -0,0 +1,50 @@
+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.IOException;
+import java.io.Closeable;
+
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.index.FieldInfo;
+
+/**
+ * @lucene.experimental
+ */
+
+public abstract class BlockTreePostingsWriterBase extends PostingsConsumer implements Closeable {
+ // nocommit
+ //public int termID;
+
+ public abstract void start(IndexOutput termsOut) throws IOException;
+
+ public abstract void startTerm() throws IOException;
+
+ /** Flush count terms starting at start "backwards", as a
+ * block. start is a negative offset from the end of the
+ * terms stack, ie bigger start means further back in
+ * the stack. */
+ public abstract void flushTermsBlock(int start, int count) throws IOException;
+
+ /** Finishes the current term */
+ public abstract void finishTerm(TermStats stats) throws IOException;
+
+ public abstract void setField(FieldInfo fieldInfo);
+
+ public abstract void close() throws IOException;
+}
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermState.java
--- /dev/null Thu Jan 01 00:00:00 1970 +0000
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermState.java Mon Aug 01 12:09:21 2011 -0400
@@ -0,0 +1,57 @@
+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 org.apache.lucene.index.DocsEnum; // javadocs
+import org.apache.lucene.index.OrdTermState;
+import org.apache.lucene.index.TermState;
+
+/**
+ * Holds all state required for {@link PostingsReaderBase}
+ * to produce a {@link DocsEnum} without re-seeking the
+ * terms dict.
+ */
+public class BlockTreeTermState extends OrdTermState {
+ public int docFreq; // how many docs have this term
+ public long totalTermFreq; // total number of occurrences of this term
+
+ 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
+
+ // nocommit -- should not be here?
+ //public int blockTermCount; // how many terms in current block
+
+ @Override
+ public void copyFrom(TermState _other) {
+ assert _other instanceof BlockTreeTermState : "can not copy from " + _other.getClass().getName();
+ BlockTreeTermState other = (BlockTreeTermState) _other;
+ super.copyFrom(_other);
+ docFreq = other.docFreq;
+ totalTermFreq = other.totalTermFreq;
+ termBlockOrd = other.termBlockOrd;
+ blockFilePointer = other.blockFilePointer;
+
+ // NOTE: don't copy blockTermCount;
+ // it's "transient": used only by the "primary"
+ // termState, and regenerated on seek by TermState
+ }
+
+ @Override
+ public String toString() {
+ return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer;
+ }
+}
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
--- /dev/null Thu Jan 01 00:00:00 1970 +0000
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java Mon Aug 01 12:09:21 2011 -0400
@@ -0,0 +1,2872 @@
+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.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.index.codecs.standard.StandardPostingsReader; // javadocs
+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.DoubleBarrelLRUCache;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
+import org.apache.lucene.util.automaton.RegExp;
+import org.apache.lucene.util.automaton.RunAutomaton;
+import org.apache.lucene.util.automaton.SpecialOperations;
+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;
+
+// nocommit
+// - need indexDivisor (?)
+
+// nocommit finish jdocs
+
+/* 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 blocks that all share a common prefix.
+ * The advantage of this approach is that {@link
+ * #seekExact} is often able to determine a term cannot
+ * exist without doing any IO. Note that this terms
+ * dictionary has it's own fixed terms index (ie, it does
+ * not support a pluggable terms index). */
+public class BlockTreeTermsReader extends FieldsProducer {
+
+ // Open input to the main terms dict file (_X.tib)
+ private final IndexInput in;
+
+ private static final int OUTPUT_FLAGS_NUM_BITS = 2;
+ private static final int OUTPUT_FLAGS_MASK = 0x3;
+ private static final int OUTPUT_FLAG_IS_FLOOR = 0x1;
+ private static final int OUTPUT_FLAG_HAS_TERMS = 0x2;
+
+ public static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
+ //private static final boolean DEBUG = false;
+
+ // Reads the terms dict entries, to gather state to
+ // produce DocsEnum on demand
+ private final BlockTreePostingsReaderBase postingsReader;
+
+ private final TreeMap fields = new TreeMap();
+
+ // Caches the most recently looked-up field + terms:
+ private final DoubleBarrelLRUCache termsCache;
+
+ private int indexDivisor;
+
+ // keeps the dirStart offset
+ protected long dirOffset;
+ protected long indexDirOffset;
+
+ // Used as key for the terms cache
+ private static class FieldAndTerm extends DoubleBarrelLRUCache.CloneableKey {
+ String field;
+ BytesRef term;
+
+ public FieldAndTerm() {
+ }
+
+ public FieldAndTerm(FieldAndTerm other) {
+ field = other.field;
+ term = new BytesRef(other.term);
+ }
+
+ @Override
+ public boolean equals(Object _other) {
+ FieldAndTerm other = (FieldAndTerm) _other;
+ return other.field == field && term.bytesEquals(other.term);
+ }
+
+ @Override
+ public Object clone() {
+ return new FieldAndTerm(this);
+ }
+
+ @Override
+ public int hashCode() {
+ return field.hashCode() * 31 + term.hashCode();
+ }
+ }
+
+ private String segment;
+
+ public BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, String segment,
+ BlockTreePostingsReaderBase postingsReader, IOContext ioContext,
+ int termsCacheSize, int codecId, int indexDivisor)
+ throws IOException {
+
+ this.postingsReader = postingsReader;
+ this.indexDivisor = indexDivisor;
+ termsCache = new DoubleBarrelLRUCache(termsCacheSize);
+
+ this.segment = segment;
+ in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTreeTermsWriter.TERMS_EXTENSION),
+ ioContext);
+
+ boolean success = false;
+ IOException ioe = null;
+ IndexInput indexIn = null;
+
+ try {
+ readHeader(in);
+ final String f = IndexFileNames.segmentFileName(segment, codecId, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION);
+ 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 long termsStartPointer = in.readVLong();
+ 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, termsStartPointer, 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) {
+ final String s;
+ try {
+ return b.utf8ToString() + " " + b;
+ } catch (Throwable t) {
+ return b.toString();
+ }
+ }
+
+ // nocommit -- have CheckIndex run this:
+ public static class BlockTreeStats {
+ 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 BlockTreeStats(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.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;
+ }
+
+ public void print(PrintStream out) {
+ out.println("BlockTree stats for segment=" + segment + " field=" + field);
+ 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();
+
+ private class FieldReader extends Terms implements Closeable {
+ final long numTerms;
+ final FieldInfo fieldInfo;
+ final long termsStartPointer;
+ 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, long termsStartPointer, 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.termsStartPointer = termsStartPointer;
+ 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() >> 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();
+ }
+
+ // nocommit -- impl terms index divisor (how!?)
+ }
+
+ // nocommit
+ if (false && fieldInfo.name.equals("body")) {
+ Automaton a = new RegExp("fa[mpty].*").toAutomaton();
+ CompiledAutomaton ca = new CompiledAutomaton(a, SpecialOperations.isFinite(a));
+ TermsEnum te = intersect(ca, null);
+ while(te.next() != null) {
+ System.out.println("m: " + te.term().utf8ToString());
+ }
+ }
+ }
+
+ @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 {
+ return new IntersectEnum(compiled, startTerm);
+ }
+
+ // NOTE: cannot seek!
+ private final class IntersectEnum extends TermsEnum {
+ private final IndexInput in;
+
+ // nocommit -- I don't need the index!?
+
+ 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();
+
+ // nocommit -- this is nearly identical to the Frame
+ // in STE!
+ private 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 BlockTreeTermState 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 {
+ if (DEBUG) System.out.println(" loadNextFoorBlock");
+ fp = fpEnd;
+
+ // Redundant check:
+ final long code = floorDataReader.readVLong();
+ final long newFP = fpOrig + (code >>> 1);
+ assert newFP == fpEnd: "newFP=" + newFP + " fpEnd=" + fpEnd;
+
+ assert numFollowFloorBlocks > 0;
+ numFollowFloorBlocks--;
+ if (numFollowFloorBlocks == 0) {
+ nextFloorLabel = 256;
+ } else {
+ nextFloorLabel = floorDataReader.readByte() & 0xFF;
+ }
+ if (DEBUG) System.out.println(" nextFloorLabel=" + nextFloorLabel);
+ 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);
+ 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;
+
+ if (frameIndexData != null && !isLastInFloor) {
+ // Floor frame
+ if (floorData.length < frameIndexData.length) {
+ this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
+ }
+ if (DEBUG) System.out.println(" frameIndexData=" + frameIndexData);
+ 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
+ floorDataReader.readVLong();
+ numFollowFloorBlocks = floorDataReader.readVInt();
+ nextFloorLabel = floorDataReader.readByte() & 0xff;
+ if (DEBUG) System.out.println(" numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
+ }
+
+ 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();
+ }
+ }
+
+ // nocommit -- maybe add a scanToLabel?
+
+ 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);
+ // nocommit -- don't incr this for leaf! same for
+ // main enum too... just set in decodeMetaData
+ termState.termBlockOrd++;
+ 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 void decodeMetaData() throws IOException {
+
+ assert termState.termBlockOrd > 0;
+
+ // lazily catch up on metadata decode:
+ final int limit = termState.termBlockOrd;
+
+ // 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++;
+ }
+ }
+ }
+
+ public IntersectEnum(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
+ // nocommit can we use suffixRef?
+ // nocommit in some cases we can do hard filter by
+ // length!! eg regexp ????????
+ runAutomaton = compiled.runAutomaton;
+ compiledAutomaton = compiled;
+ in = (IndexInput) BlockTreeTermsReader.this.in.clone();
+ stack = new Frame[5];
+ for(int idx=0;idx();
+ }
+ 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);
+
+ 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;
+ 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;
+ // nocommit -- not efficient; caller should
+ // provide the first step in the arc; else we do
+ // O(n^2) here!
+ 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 {
+ //System.out.println("BTR.docFreq");
+ currentFrame.decodeMetaData();
+ //System.out.println(" return " + state.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 final static boolean DEBUG_SEEK1 = false;
+
+ // NOTE: specialized to only doing the first-time
+ // seek, but we could generalize it to allow
+ // arbitrary seekExact/Ceil.
+ private void seekToStartTerm(BytesRef target) throws IOException {
+ if (DEBUG_SEEK1) System.out.println("seek to startTerm=" + target.utf8ToString());
+ assert currentFrame.ord == 0;
+ if (term.length < target.length) {
+ term.bytes = ArrayUtil.grow(term.bytes, target.length);
+ }
+ FST.Arc arc = arcs[0];
+ assert arc == currentFrame.arc;
+
+ for(int idx=0;idx getComparator() {
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
+ }
+
+ @Override
+ public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void seekExact(long ord) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public long ord() throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ // Iterates through terms in this field
+ private final class SegmentTermsEnum extends TermsEnum {
+ private final IndexInput in;
+ private final FieldAndTerm fieldTerm = new FieldAndTerm();
+
+ private Frame[] stack;
+ private final Frame staticFrame;
+ private Frame currentFrame;
+ private boolean termExists;
+
+ private int targetBeforeCurrentLength;
+
+ private final ByteArrayDataInput scratchReader = new ByteArrayDataInput();
+
+ // What prefix of the current term was present in the index:
+ private int validIndexPrefix;
+
+ // assert only:
+ private boolean eof;
+
+ final BytesRef term = new BytesRef();
+
+ @SuppressWarnings("unchecked") private FST.Arc[] arcs = new FST.Arc[5];
+
+ public SegmentTermsEnum() throws IOException {
+ if (DEBUG) System.out.println("BTTR.init seg=" + segment);
+ in = (IndexInput) BlockTreeTermsReader.this.in.clone();
+ in.seek(termsStartPointer);
+ fieldTerm.field = fieldInfo.name;
+ 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();
+
+ // nocommit -- can we avoid this? eg if is going to
+ // call seek...
+ 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 BlockTreeStats computeBlockStats() throws IOException {
+
+ BlockTreeStats stats = new BlockTreeStats(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 >> OUTPUT_FLAGS_NUM_BITS;
+ final Frame f = getFrame(1+currentFrame.ord);
+ f.hasTerms = (code & OUTPUT_FLAG_HAS_TERMS) != 0;
+ f.hasTermsOrig = f.hasTerms;
+ f.isFloor = (code & 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 + ") useCache=" + useCache + " validIndexPrefix=" + validIndexPrefix);
+ printSeekState();
+ }
+
+ // Check cache
+ if (useCache) {
+ fieldTerm.term = target;
+ // TODO: should we differentiate "frozen"
+ // TermState (ie one that was cloned and
+ // cached/returned by termState()) from the
+ // malleable (primary) one?
+ final TermState cachedState = termsCache.get(fieldTerm);
+ if (cachedState != null) {
+ if (DEBUG) {
+ System.out.println(" cached!");
+ }
+ seekExact(target, cachedState);
+ //System.out.println(" term=" + term.utf8ToString());
+ return true;
+ }
+ // nocommit -- we never enroll state into the termsCache!
+ }
+
+ FST.Arc arc;
+ int targetUpto;
+ BytesRef output;
+
+ // nocommit: use FST's root arc cache somehow? oh
+ // we do already -- verify the lookup is using it!
+
+ 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;
+
+ // nocommit try reversing vLong byte order!!
+
+ // nocommit test empty string seeking when we have
+ // seek state
+
+ // 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:
+ 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));
+ }
+
+ // nocommit right? or am i losing seek reuse!!
+ // what if targetUpto is 0!?
+ // nocommit +1?
+ validIndexPrefix = currentFrame.prefix;
+ //validIndexPrefix = targetUpto;
+
+ currentFrame.scanToFloorFrame(target);
+
+ if (!currentFrame.hasTerms) {
+ termExists = false;
+ // nocommit -- should we just set length here?
+ // nocommit -- only have to copy suffix:
+ term.bytes[targetUpto] = (byte) targetLabel;
+ term.length = 1+targetUpto;
+ //term.copy(target);
+ //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 (result == SeekStatus.END) {
+ // nocommit -- merge w/ else clause
+ if (DEBUG) {
+ System.out.println(" return NOT_FOUND term=" + brToString(term));
+ }
+ return false;
+ } else {
+ if (DEBUG) {
+ System.out.println(" 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;
+ // nocommit -- should we just set length here?
+ // nocommit -- only have to copy suffix:
+ //term.copy(target);
+ 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 (result == SeekStatus.END) {
+ // nocommit -- merge w/ else clause
+ if (DEBUG) {
+ System.out.println(" return NOT_FOUND term=" + brToString(term));
+ }
+ return false;
+ } else {
+ //termExists = false;
+ if (DEBUG) {
+ System.out.println(" return NOT_FOUND term=" + term.utf8ToString());
+ }
+
+ return false;
+ }
+
+ // nocommit -- add back asserts that verify we don't
+ // scan too many blocks...
+ }
+
+ @Override
+ public SeekStatus seekCeil(final BytesRef target, final boolean useCache) throws IOException {
+ // nocommit can this be an assert...?
+ 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 + ") useCache=" + useCache + " validIndexPrefix= " + validIndexPrefix);
+ //printSeekState();
+ //}
+
+ // Check cache
+ if (useCache) {
+ fieldTerm.term = target;
+ // TODO: should we differentiate "frozen"
+ // TermState (ie one that was cloned and
+ // cached/returned by termState()) from the
+ // malleable (primary) one?
+ final TermState cachedState = termsCache.get(fieldTerm);
+ if (cachedState != null) {
+ //if (DEBUG) {
+ //System.out.println(" cached!");
+ //}
+ seekExact(target, cachedState);
+ //System.out.println(" term=" + term.utf8ToString());
+ return SeekStatus.FOUND;
+ }
+ // nocommit -- we never enroll state into the termsCache!
+ }
+
+ 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;
+
+ // nocommit try reversing vLong byte order!!
+
+ // nocommit test empty string seeking when we have
+ // seek state
+
+ // 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);
+ // nocommit -- we could save the outputs in
+ // local byte[][]?
+ 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));
+ }
+
+ // nocommit right? or am i losing seek reuse!!
+ // what if targetUpto is 0!?
+ // nocommit -- this differs from seekExact!?
+ validIndexPrefix = currentFrame.prefix;
+ //validIndexPrefix = targetUpto;
+
+ currentFrame.scanToFloorFrame(target);
+
+ currentFrame.loadBlock();
+
+ final SeekStatus result = currentFrame.scanToTerm(target, false);
+ if (result == SeekStatus.END) {
+
+ // nocommit -- these 2 aren't needed?
+ 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) {
+ // nocommit?
+ 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;
+ }
+
+ // nocommit -- add back asserts that verify we don't
+ // scan too many blocks...
+ }
+
+ 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) {
+ // nocommit -- this is wasteful, if it's a
+ // floor block and we are gonna move fp back
+ // to the loaded fp in scanToFloorFrame; in
+ // this case we re-seek unnecessarily
+ 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;
+ // nocommit: only if hasTerms?
+ 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;
+ state.termBlockOrd++;
+ 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) {
+ // nocommit -- instead of taking target, can this
+ // just use term?
+
+ 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;
+ hasTerms = hasTermsOrig;
+ while (true) {
+ final long code = floorDataReader.readVLong();
+ hasTerms = (code & 1) != 0;
+ newFP = fpOrig + (code >>> 1);
+ //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);
+
+ assert state.termBlockOrd > 0;
+
+ // lazily catch up on metadata decode:
+ final int limit = state.termBlockOrd;
+
+ // 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) {
+
+ blockMDScanCount++;
+
+ // 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;
+
+ // nocommit should take min block size?
+ public BlockTreeTermsWriter(
+ SegmentWriteState state,
+ BlockTreePostingsWriterBase postingsWriter,
+ int minItemsInBlock,
+ int maxItemsInBlock)
+ throws IOException
+ {
+
+ // nocommit -- make sure minItemsInBlock is > 1
+
+ if (minItemsInBlock <= 0) {
+ throw new IllegalArgumentException("minItemsInBlock must be >= 1; 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 (DEBUG2 || 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;
+ }
+
+ private static class PendingTerm {
+ public final BytesRef term;
+ public final TermStats stats;
+
+ public PendingTerm(BytesRef term, TermStats stats) {
+ this.term = term;
+ this.stats = stats;
+ }
+
+ @Override
+ public String toString() {
+ return term.utf8ToString();
+ }
+ }
+
+ static long encodeOutput(long fp, boolean hasTerms, boolean isFloor) {
+ // nocommit assert fp is "small enough"
+ // nocommit use constants here instead of 1, 2:
+ return (fp << 2) | (hasTerms ? 2 : 0) | (isFloor ? 1 : 0);
+ }
+
+ private static class PendingBlock {
+ 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) {
+ 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;
+ // nocommit -- vLong is bad for FST!!! it writes LSB
+ // first which means less byte[] prefix sharing I
+ // think??? sheesh.
+ 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;
+ // nocommit -- why do we need hasTerms here?
+ // nocommit -- need isFloor here?
+ 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, true, 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
+
+ for(FST subIndex : subIndices) {
+ append(indexBuilder, subIndex);
+ }
+
+ if (floorBlocks != null) {
+ for (PendingBlock sub : floorBlocks) {
+ 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 final long termsStartPointer;
+ 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