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: + *

    + *
  1. As long as matches are successful, keep reading sequentially. + *
  2. 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 blockBuilder; + + // PendingTerm or PendingBlock: + private final List pending = new ArrayList(); + + // This class assigns terms to blocks "naturally", ie, + // according to the number of terms under a given prefix + // that we encounter: + private class FindBlocks extends Builder.FreezeTail { + + @Override + public void freeze(final Builder.UnCompiledNode[] frontier, int prefixLenPlus1, final IntsRef lastInput) throws IOException { + + if (DEBUG) System.out.println(" freeze prefixLenPlus1=" + prefixLenPlus1); + + for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) { + final Builder.UnCompiledNode node = frontier[idx]; + final Builder.UnCompiledNode parent = idx == 0 ? null : frontier[idx-1]; + + long totCount = 0; + + if (node.isFinal) { + totCount++; + } + + //System.out.println("VISIT node=" + node + " + //arcs=" + node.numArcs); + for(int arcIdx=0;arcIdx target = (Builder.UnCompiledNode) node.arcs[arcIdx].target; + totCount += target.inputCount; + target.clear(); + node.arcs[arcIdx].target = null; + } + node.numArcs = 0; + + boolean forceBlock = false; + + // nocommit fixup + + if (idx == 1) { + // nocommit -- make this 1 configurable -- maybe + // 2 is better if there are many terms? + + // We force a block if prefix is length 1 and + // there are any terms, so that the root block + // doesn't have terms. + + // nocommit: instead, we should accum termCount & + // blockCount into UnCompiledNode? + for(int pendingIdx=0;pendingIdx= minItemsInBlock || idx == 0 || forceBlock) { + if (DEBUG2 || DEBUG) { + if (totCount < minItemsInBlock && idx != 0) { + System.out.println(" force block has terms"); + } + } + node.inputCount = writeBlocks(lastInput, idx, (int) totCount); + } else { + // stragglers! carry count upwards + node.inputCount = totCount; + } + frontier[idx] = new Builder.UnCompiledNode(blockBuilder, idx); + } + } + } + + private int[] subBytes = new int[10]; + private int[] subTermCounts = new int[10]; + private int[] subTermCountSums = new int[10]; + private int[] subSubCounts = new int[10]; + + // Write the top count entries on the pending stack as + // one or more blocks. + int writeBlocks(IntsRef prevTerm, int prefixLength, int count) throws IOException { + if (prefixLength == 0 || count <= maxItemsInBlock) { + // Not floor block + final PendingBlock nonFloorBlock = writeBlock(prevTerm, prefixLength, prefixLength, count, count, 0, false, -1, true); + nonFloorBlock.compileIndex(null, scratchBytes); + pending.add(nonFloorBlock); + } else { + + // nocommit -- we could enrich this format so that + // we store min & max label for this block, then it + // can be "authoritative" + + if (DEBUG) { + final BytesRef prefix = new BytesRef(prefixLength); + for(int m=0;m 1: "subCount=" + subCount + " sub=" + sub + " of " + numSubs + " subTermCount=" + subTermCountSums[sub] + " subSubCount=" + subSubCounts[sub] + " depth=" + prefixLength; + subCount = 0; + startLabel = subBytes[sub+1]; + + if (curStart == 0) { + break; + } + + if (curStart <= maxItemsInBlock) { + // remainder is small enough to fit into a + // block. NOTE that this may be too small (< + // minItemsInBlock); need a true segmenter + // here + assert startLabel != -1; + assert firstBlock != null; + prevTerm.ints[prevTerm.offset + prefixLength] = startLabel; + //System.out.println(" final " + (numSubs-sub-1) + " subs"); + floorBlocks.add(writeBlock(prevTerm, prefixLength, prefixLength+1, curStart, curStart, 0, true, startLabel, true)); + break; + } + } + } + + prevTerm.ints[prevTerm.offset + prefixLength] = savLabel; + + assert firstBlock != null; + firstBlock.compileIndex(floorBlocks, scratchBytes); + + pending.add(firstBlock); + if (DEBUG) System.out.println(" done pending.size()=" + pending.size()); + } + + return 1; + } + + String brPrefixToString(BytesRef b) { + // nocommit + return b.toString(); + //return b.utf8ToString() + " " + b; + } + + String brToString(BytesRef b) { + // nocommit + // return b.toString(); + return b.utf8ToString() + " " + b; + } + + // TODO: we could block-write the term suffix pointers; + // this would take more space but would enable binary + // search on lookup + private PendingBlock writeBlock(IntsRef prevTerm, int prefixLength, int indexPrefixLength, int start, int length, int futureTermCount, boolean isFloor, int floorLeadByte, boolean isLastInFloor) throws IOException { + + assert length > 0; + + final BytesRef prefix = new BytesRef(indexPrefixLength); + for(int m=0;m(FST.INPUT_TYPE.BYTE1, + 0, 0, true, + true, Integer.MAX_VALUE, + noOutputs, + new FindBlocks()); + + termsStartPointer = out.getFilePointer(); + postingsWriter.setField(fieldInfo); + } + + @Override + public Comparator getComparator() { + return BytesRef.getUTF8SortedAsUnicodeComparator(); + } + + @Override + public PostingsConsumer startTerm(BytesRef text) throws IOException { + //System.out.println("BTW.startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment); + postingsWriter.startTerm(); + /* + if (fieldInfo.name.equals("id")) { + postingsWriter.termID = Integer.parseInt(text.utf8ToString()); + } else { + postingsWriter.termID = -1; + } + */ + return postingsWriter; + } + + @Override + public void finishTerm(BytesRef text, TermStats stats) throws IOException { + + assert stats.docFreq > 0; + if (DEBUG) System.out.println("BTTW.finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq); + + blockBuilder.add(text, noOutputs.getNoOutput()); + pending.add(new PendingTerm(new BytesRef(text), stats)); + postingsWriter.finishTerm(stats); + numTerms++; + } + + // Finishes all terms in this field + @Override + public void finish(long sumTotalTermFreq, long sumDocFreq) throws IOException { + // nocommit write sumDocFreq + if (numTerms > 0) { + blockBuilder.finish(); + + // We better have one final "root" block: + assert pending.size() == 1 && pending.get(0) instanceof PendingBlock: "pending.size()=" + pending.size() + " pending=" + pending; + final PendingBlock root = (PendingBlock) pending.get(0); + assert root.prefix.length == 0; + assert root.index.getEmptyOutput() != null; + + this.sumTotalTermFreq = sumTotalTermFreq; + this.sumDocFreq = sumDocFreq; + + // Write FST to index + indexStartFP = indexOut.getFilePointer(); + root.index.save(indexOut); + //System.out.println(" write FST " + indexStartFP + " field=" + fieldInfo.name); + + if (SAVE_DOT_FILES || DEBUG2 || DEBUG) { + final String dotFileName = segment + "_" + fieldInfo.name + ".dot"; + Writer w = new OutputStreamWriter(new FileOutputStream(dotFileName)); + Util.toDot(root.index, w, false, false); + System.out.println("SAVED to " + dotFileName); + w.close(); + } + } + } + + private final RAMOutputStream bytesWriter = new RAMOutputStream(); + } + + @Override + public void close() throws IOException { + + IOException ioe = null; + try { + + int nonZeroCount = 0; + for(TermsWriter field : fields) { + if (field.numTerms > 0) { + nonZeroCount++; + } + } + + final long dirStart = out.getFilePointer(); + final long indexDirStart = indexOut.getFilePointer(); + + out.writeVInt(nonZeroCount); + + for(TermsWriter field : fields) { + if (field.numTerms > 0) { + //System.out.println(" field " + field.fieldInfo.name + " " + field.numTerms + " terms"); + out.writeVInt(field.fieldInfo.number); + out.writeVLong(field.numTerms); + // nocommit: we may not need termsStartPointer? + out.writeVLong(field.termsStartPointer); + final BytesRef rootCode = ((PendingBlock) field.pending.get(0)).index.getEmptyOutput(); + assert rootCode != null: "field=" + field.fieldInfo.name + " numTerms=" + field.numTerms; + out.writeVInt(rootCode.length); + out.writeBytes(rootCode.bytes, rootCode.offset, rootCode.length); + if (field.fieldInfo.indexOptions != IndexOptions.DOCS_ONLY) { + out.writeVLong(field.sumTotalTermFreq); + } + out.writeVLong(field.sumDocFreq); + indexOut.writeVLong(field.indexStartFP); + } + } + writeTrailer(dirStart); + writeIndexTrailer(indexDirStart); + } catch (IOException ioe2) { + ioe = ioe2; + } finally { + IOUtils.closeSafely(ioe, out, indexOut, postingsWriter); + } + } +} Index: lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java --- lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Mon Aug 01 12:09:21 2011 -0400 @@ -43,7 +43,7 @@ private final Set knownExtensions = new HashSet(); - public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText", "Memory"}; + public final static String[] CORE_CODECS = new String[] {"Standard", "StandardTree", "Pulsing", "PulsingTree", "PreFlex", "SimpleText", "Memory"}; public synchronized void register(Codec codec) { if (codec.name == null) { @@ -84,7 +84,7 @@ public synchronized Codec lookup(String name) { final Codec codec = codecs.get(name); if (codec == null) { - throw new IllegalArgumentException("required codec '" + name + "' not found"); + throw new IllegalArgumentException("required codec '" + name + "' not found; known codecs: " + codecs.keySet()); } return codec; } Index: lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java --- lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/index/codecs/CoreCodecProvider.java Mon Aug 01 12:09:21 2011 -0400 @@ -20,8 +20,10 @@ import org.apache.lucene.index.codecs.memory.MemoryCodec; import org.apache.lucene.index.codecs.preflex.PreFlexCodec; import org.apache.lucene.index.codecs.pulsing.PulsingCodec; +import org.apache.lucene.index.codecs.pulsingtree.PulsingTreeCodec; import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec; import org.apache.lucene.index.codecs.standard.StandardCodec; +import org.apache.lucene.index.codecs.standardtree.StandardTreeCodec; /** * A CodecProvider that registers all core codecs that ship @@ -43,8 +45,11 @@ public class CoreCodecProvider extends CodecProvider { public CoreCodecProvider() { register(new StandardCodec()); + register(new StandardTreeCodec(25, 48)); register(new PreFlexCodec()); register(new PulsingCodec()); + // nocommit: how come no args to this one? + register(new PulsingTreeCodec(1)); register(new SimpleTextCodec()); register(new MemoryCodec()); } Index: lucene/src/java/org/apache/lucene/index/codecs/pulsingtree/PulsingTreeCodec.java --- /dev/null Thu Jan 01 00:00:00 1970 +0000 +++ lucene/src/java/org/apache/lucene/index/codecs/pulsingtree/PulsingTreeCodec.java Mon Aug 01 12:09:21 2011 -0400 @@ -0,0 +1,141 @@ +package org.apache.lucene.index.codecs.pulsingtree; + +/** + * 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.Set; + +import org.apache.lucene.index.PerDocWriteState; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.codecs.BlockTreePostingsReaderBase; +import org.apache.lucene.index.codecs.BlockTreePostingsWriterBase; +import org.apache.lucene.index.codecs.BlockTreeTermsReader; +import org.apache.lucene.index.codecs.BlockTreeTermsWriter; +import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.codecs.DefaultDocValuesConsumer; +import org.apache.lucene.index.codecs.DefaultDocValuesProducer; +import org.apache.lucene.index.codecs.FieldsConsumer; +import org.apache.lucene.index.codecs.FieldsProducer; +import org.apache.lucene.index.codecs.PerDocConsumer; +import org.apache.lucene.index.codecs.PerDocValues; +import org.apache.lucene.index.codecs.standardtree.StandardTreeCodec; +import org.apache.lucene.index.codecs.standardtree.StandardTreePostingsReader; +import org.apache.lucene.index.codecs.standardtree.StandardTreePostingsWriter; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; + +/** This codec "inlines" the postings for terms that have + * low docFreq. It wraps another codec, which is used for + * writing the non-inlined terms. + * + * Currently in only inlines docFreq=1 terms, and + * otherwise uses the normal "standard" codec. + * @lucene.experimental */ + +public class PulsingTreeCodec extends Codec { + + private final int freqCutoff; + + /** Terms with freq <= freqCutoff are inlined into terms + * dict. */ + public PulsingTreeCodec(int freqCutoff) { + super("PulsingTree"); + this.freqCutoff = freqCutoff; + } + + @Override + public String toString() { + return name + "(freqCutoff=" + freqCutoff + ")"; + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + // We wrap StandardTreePostingsWriter, but any BlockTreePostingsWriterBase + // will work: + + BlockTreePostingsWriterBase docsWriter = new StandardTreePostingsWriter(state); + + // Terms that have <= freqCutoff number of docs are + // "pulsed" (inlined): + BlockTreePostingsWriterBase pulsingWriter = new PulsingTreePostingsWriter(freqCutoff, docsWriter); + + // Terms dict + boolean success = false; + try { + // nocommit make this 24 configurable + FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, 32, 64); + success = true; + return ret; + } finally { + if (!success) { + pulsingWriter.close(); + } + } + } + + @Override + public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { + + // We wrap StandardTreePostingsReader, but any BlockTreeStandardPostingsReader + // will work: + BlockTreePostingsReaderBase docsReader = new StandardTreePostingsReader(state.dir, state.segmentInfo, state.context, state.codecId); + BlockTreePostingsReaderBase pulsingReader = new PulsingTreePostingsReader(docsReader); + + boolean success = false; + try { + FieldsProducer ret = new BlockTreeTermsReader( + state.dir, state.fieldInfos, state.segmentInfo.name, + pulsingReader, + state.context, + StandardTreeCodec.TERMS_CACHE_SIZE, + state.codecId, + state.termsIndexDivisor); + success = true; + return ret; + } finally { + if (!success) { + pulsingReader.close(); + } + } + } + + @Override + public void files(Directory dir, SegmentInfo segmentInfo, int codecID, Set files) throws IOException { + StandardTreePostingsReader.files(dir, segmentInfo, codecID, files); + BlockTreeTermsReader.files(dir, segmentInfo, codecID, files); + DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS()); + } + + @Override + public void getExtensions(Set extensions) { + StandardTreeCodec.getStandardExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS()); + } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context); + } +} Index: lucene/src/java/org/apache/lucene/index/codecs/pulsingtree/PulsingTreePostingsReader.java --- /dev/null Thu Jan 01 00:00:00 1970 +0000 +++ lucene/src/java/org/apache/lucene/index/codecs/pulsingtree/PulsingTreePostingsReader.java Mon Aug 01 12:09:21 2011 -0400 @@ -0,0 +1,498 @@ +package org.apache.lucene.index.codecs.pulsingtree; + +/** + * 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 org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.TermState; +import org.apache.lucene.index.codecs.BlockTreePostingsReaderBase; +import org.apache.lucene.index.codecs.BlockTreeTermState; +import org.apache.lucene.store.ByteArrayDataInput; +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; + +/** Concrete class that reads the current doc/freq/skip + * postings format + * @lucene.experimental */ + +// TODO: -- should we switch "hasProx" higher up? and +// create two separate docs readers, one that also reads +// prox and one that doesn't? + +public class PulsingTreePostingsReader extends BlockTreePostingsReaderBase { + + // Fallback reader for non-pulsed terms: + final BlockTreePostingsReaderBase wrappedPostingsReader; + int maxPositions; + + public PulsingTreePostingsReader(BlockTreePostingsReaderBase wrappedPostingsReader) throws IOException { + this.wrappedPostingsReader = wrappedPostingsReader; + } + + @Override + public void init(IndexInput termsIn) throws IOException { + CodecUtil.checkHeader(termsIn, PulsingTreePostingsWriter.CODEC, + PulsingTreePostingsWriter.VERSION_START, PulsingTreePostingsWriter.VERSION_START); + maxPositions = termsIn.readVInt(); + wrappedPostingsReader.init(termsIn); + } + + private static class PulsingTermState extends BlockTreeTermState { + private byte[] postings; + private int postingsSize; // -1 if this term was not inlined + private BlockTreeTermState wrappedTermState; + + ByteArrayDataInput inlinedBytesReader; + private byte[] inlinedBytes; + + @Override + public Object clone() { + PulsingTermState clone; + clone = (PulsingTermState) super.clone(); + if (postingsSize != -1) { + clone.postings = new byte[postingsSize]; + System.arraycopy(postings, 0, clone.postings, 0, postingsSize); + } else { + assert wrappedTermState != null; + clone.wrappedTermState = (BlockTreeTermState) wrappedTermState.clone(); + } + return clone; + } + + @Override + public void copyFrom(TermState _other) { + super.copyFrom(_other); + PulsingTermState other = (PulsingTermState) _other; + postingsSize = other.postingsSize; + if (other.postingsSize != -1) { + if (postings == null || postings.length < other.postingsSize) { + postings = new byte[ArrayUtil.oversize(other.postingsSize, 1)]; + } + System.arraycopy(other.postings, 0, postings, 0, other.postingsSize); + } else { + wrappedTermState.copyFrom(other.wrappedTermState); + } + + // NOTE: we do not copy the + // inlinedBytes/inlinedBytesReader; these are only + // stored on the "primary" TermState. They are + // "transient" to cloned term states. + } + + @Override + public String toString() { + if (postingsSize == -1) { + return "PulsingTermState: not inlined: wrapped=" + wrappedTermState; + } else { + return "PulsingTermState: inlined size=" + postingsSize + " " + super.toString(); + } + } + } + + @Override + public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTreeTermState _termState) throws IOException { + //System.out.println("PR.readTermsBlock"); + final PulsingTermState termState = (PulsingTermState) _termState; + if (termState.inlinedBytes == null) { + termState.inlinedBytes = new byte[128]; + termState.inlinedBytesReader = new ByteArrayDataInput(); + } + int len = termsIn.readVInt(); + //System.out.println(" len=" + len + " fp=" + termsIn.getFilePointer()); + if (termState.inlinedBytes.length < len) { + termState.inlinedBytes = new byte[ArrayUtil.oversize(len, 1)]; + } + termsIn.readBytes(termState.inlinedBytes, 0, len); + termState.inlinedBytesReader.reset(termState.inlinedBytes); + termState.wrappedTermState.termBlockOrd = 0; + wrappedPostingsReader.readTermsBlock(termsIn, fieldInfo, termState.wrappedTermState); + } + + @Override + public void resetTermsBlock(FieldInfo fieldInfo, BlockTreeTermState _termState) throws IOException { + final PulsingTermState termState = (PulsingTermState) _termState; + if (termState.inlinedBytes != null) { + termState.inlinedBytesReader.rewind(); + } + termState.wrappedTermState.termBlockOrd = 0; + wrappedPostingsReader.resetTermsBlock(fieldInfo, termState.wrappedTermState); + } + + @Override + public BlockTreeTermState newTermState() throws IOException { + PulsingTermState state = new PulsingTermState(); + state.wrappedTermState = wrappedPostingsReader.newTermState(); + return state; + } + + @Override + public void nextTerm(FieldInfo fieldInfo, BlockTreeTermState _termState) throws IOException { + //System.out.println("PR nextTerm"); + PulsingTermState termState = (PulsingTermState) _termState; + + // total TF, but in the omitTFAP case its computed based + // on docFreq. + long count = fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS ? termState.totalTermFreq : termState.docFreq; + //System.out.println(" count=" + count + " threshold=" + maxPositions); + + if (count <= maxPositions) { + + // Inlined into terms dict -- just read the byte[] blob in, + // but don't decode it now (we only decode when a DocsEnum + // or D&PEnum is pulled): + termState.postingsSize = termState.inlinedBytesReader.readVInt(); + if (termState.postings == null || termState.postings.length < termState.postingsSize) { + termState.postings = new byte[ArrayUtil.oversize(termState.postingsSize, 1)]; + } + // TODO: sort of silly to copy from one big byte[] + // (the blob holding all inlined terms' blobs for + // current term block) into another byte[] (just the + // blob for this term)... + termState.inlinedBytesReader.readBytes(termState.postings, 0, termState.postingsSize); + //System.out.println(" inlined bytes=" + termState.postingsSize); + } else { + //System.out.println(" not inlined"); + termState.postingsSize = -1; + // TODO: should we do full copyFrom? much heavier...? + termState.wrappedTermState.docFreq = termState.docFreq; + termState.wrappedTermState.totalTermFreq = termState.totalTermFreq; + wrappedPostingsReader.nextTerm(fieldInfo, termState.wrappedTermState); + termState.wrappedTermState.termBlockOrd++; + } + } + + // TODO: we could actually reuse, by having TL that + // holds the last wrapped reuse, and vice-versa + @Override + public DocsEnum docs(FieldInfo field, BlockTreeTermState _termState, Bits liveDocs, DocsEnum reuse) throws IOException { + PulsingTermState termState = (PulsingTermState) _termState; + if (termState.postingsSize != -1) { + PulsingDocsEnum postings; + if (reuse instanceof PulsingDocsEnum) { + postings = (PulsingDocsEnum) reuse; + if (!postings.canReuse(field)) { + postings = new PulsingDocsEnum(field); + } + } else { + postings = new PulsingDocsEnum(field); + } + return postings.reset(liveDocs, termState); + } else { + // TODO: not great that we lose reuse of PulsingDocsEnum in this case: + if (reuse instanceof PulsingDocsEnum) { + return wrappedPostingsReader.docs(field, termState.wrappedTermState, liveDocs, null); + } else { + return wrappedPostingsReader.docs(field, termState.wrappedTermState, liveDocs, reuse); + } + } + } + + // TODO: -- not great that we can't always reuse + @Override + public DocsAndPositionsEnum docsAndPositions(FieldInfo field, BlockTreeTermState _termState, Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException { + if (field.indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + return null; + } + //System.out.println("D&P: field=" + field.name); + + final PulsingTermState termState = (PulsingTermState) _termState; + + if (termState.postingsSize != -1) { + PulsingDocsAndPositionsEnum postings; + if (reuse instanceof PulsingDocsAndPositionsEnum) { + postings = (PulsingDocsAndPositionsEnum) reuse; + if (!postings.canReuse(field)) { + postings = new PulsingDocsAndPositionsEnum(field); + } + } else { + postings = new PulsingDocsAndPositionsEnum(field); + } + + return postings.reset(liveDocs, termState); + } else { + if (reuse instanceof PulsingDocsAndPositionsEnum) { + return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, null); + } else { + return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, reuse); + } + } + } + + private static class PulsingDocsEnum extends DocsEnum { + private final ByteArrayDataInput postings = new ByteArrayDataInput(); + private final IndexOptions indexOptions; + private final boolean storePayloads; + private Bits liveDocs; + private int docID; + private int freq; + + public PulsingDocsEnum(FieldInfo fieldInfo) { + indexOptions = fieldInfo.indexOptions; + storePayloads = fieldInfo.storePayloads; + } + + public PulsingDocsEnum reset(Bits liveDocs, PulsingTermState termState) { + //System.out.println("PR docsEnum termState=" + termState + " docFreq=" + termState.docFreq); + assert termState.postingsSize != -1; + // nocommit -- reuse the last byte[] if we can? or + // can we directly ref termState's bytes...? dangerous? + final byte[] bytes = new byte[termState.postingsSize]; + System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize); + postings.reset(bytes); + docID = 0; + freq = 1; + this.liveDocs = liveDocs; + return this; + } + + boolean canReuse(FieldInfo fieldInfo) { + return indexOptions == fieldInfo.indexOptions && storePayloads == fieldInfo.storePayloads; + } + + @Override + public int nextDoc() throws IOException { + //System.out.println("PR nextDoc this= "+ this); + while(true) { + if (postings.eof()) { + //System.out.println("PR END"); + return docID = NO_MORE_DOCS; + } + + final int code = postings.readVInt(); + //System.out.println(" read code=" + code); + if (indexOptions == IndexOptions.DOCS_ONLY) { + docID += code; + } else { + docID += code >>> 1; // shift off low bit + if ((code & 1) != 0) { // if low bit is set + freq = 1; // freq is one + } else { + freq = postings.readVInt(); // else read freq + } + + // Skip positions + if (storePayloads) { + int payloadLength = -1; + for(int pos=0;pos= target) + return doc; + } + return docID = NO_MORE_DOCS; + } + } + + private static class PulsingDocsAndPositionsEnum extends DocsAndPositionsEnum { + private final ByteArrayDataInput postings = new ByteArrayDataInput(); + private final boolean storePayloads; + + private Bits liveDocs; + private int docID; + private int freq; + private int posPending; + private int position; + private int payloadLength; + private BytesRef payload; + + private boolean payloadRetrieved; + + public PulsingDocsAndPositionsEnum(FieldInfo fieldInfo) { + storePayloads = fieldInfo.storePayloads; + } + + boolean canReuse(FieldInfo fieldInfo) { + return storePayloads == fieldInfo.storePayloads; + } + + public PulsingDocsAndPositionsEnum reset(Bits liveDocs, PulsingTermState termState) { + assert termState.postingsSize != -1; + final byte[] bytes = new byte[termState.postingsSize]; + System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize); + postings.reset(bytes); + this.liveDocs = liveDocs; + payloadLength = 0; + docID = 0; + //System.out.println("PR d&p reset storesPayloads=" + storePayloads + " bytes=" + bytes.length + " this=" + this); + return this; + } + + @Override + public int nextDoc() throws IOException { + //System.out.println("PR d&p nextDoc this=" + this); + + while(true) { + //System.out.println(" cycle skip posPending=" + posPending); + + skipPositions(); + + if (postings.eof()) { + //System.out.println("PR END"); + return docID = NO_MORE_DOCS; + } + + final int code = postings.readVInt(); + docID += code >>> 1; // shift off low bit + if ((code & 1) != 0) { // if low bit is set + freq = 1; // freq is one + } else { + freq = postings.readVInt(); // else read freq + } + posPending = freq; + + if (liveDocs == null || liveDocs.get(docID)) { + //System.out.println(" return docID=" + docID + " freq=" + freq); + position = 0; + return docID; + } + } + } + + @Override + public int freq() { + return freq; + } + + @Override + public int docID() { + return docID; + } + + @Override + public int advance(int target) throws IOException { + int doc; + while((doc=nextDoc()) != NO_MORE_DOCS) { + if (doc >= target) { + return doc; + } + } + return docID = NO_MORE_DOCS; + } + + @Override + public int nextPosition() throws IOException { + //System.out.println("PR d&p nextPosition posPending=" + posPending + " vs freq=" + freq); + + assert posPending > 0; + posPending--; + + if (storePayloads) { + if (!payloadRetrieved) { + //System.out.println("PR skip payload=" + payloadLength); + postings.skipBytes(payloadLength); + } + final int code = postings.readVInt(); + //System.out.println("PR code=" + code); + if ((code & 1) != 0) { + payloadLength = postings.readVInt(); + //System.out.println("PR new payload len=" + payloadLength); + } + position += code >> 1; + payloadRetrieved = false; + } else { + position += postings.readVInt(); + } + + //System.out.println("PR d&p nextPos return pos=" + position + " this=" + this); + return position; + } + + private void skipPositions() throws IOException { + while(posPending != 0) { + nextPosition(); + } + if (storePayloads && !payloadRetrieved) { + //System.out.println(" skip payload len=" + payloadLength); + postings.skipBytes(payloadLength); + payloadRetrieved = true; + } + } + + @Override + public boolean hasPayload() { + return storePayloads && !payloadRetrieved && payloadLength > 0; + } + + @Override + public BytesRef getPayload() throws IOException { + //System.out.println("PR getPayload payloadLength=" + payloadLength + " this=" + this); + if (payloadRetrieved) { + throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once."); + } + payloadRetrieved = true; + if (payloadLength > 0) { + if (payload == null) { + payload = new BytesRef(payloadLength); + } else { + payload.grow(payloadLength); + } + postings.readBytes(payload.bytes, 0, payloadLength); + payload.length = payloadLength; + return payload; + } else { + return null; + } + } + } + + @Override + public void close() throws IOException { + wrappedPostingsReader.close(); + } +} Index: lucene/src/java/org/apache/lucene/index/codecs/pulsingtree/PulsingTreePostingsWriter.java --- /dev/null Thu Jan 01 00:00:00 1970 +0000 +++ lucene/src/java/org/apache/lucene/index/codecs/pulsingtree/PulsingTreePostingsWriter.java Mon Aug 01 12:09:21 2011 -0400 @@ -0,0 +1,380 @@ +package org.apache.lucene.index.codecs.pulsingtree; + +/** + * 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.List; +import java.util.ArrayList; + +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.codecs.BlockTreePostingsWriterBase; +import org.apache.lucene.index.codecs.BlockTreeTermsWriter; +import org.apache.lucene.index.codecs.TermStats; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RAMOutputStream; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CodecUtil; + +// TODO: we now inline based on total TF of the term, +// but it might be better to inline by "net bytes used" +// so that a term that has only 1 posting but a huge +// payload would not be inlined. Though this is +// presumably rare in practice... + +/** @lucene.experimental */ +public final class PulsingTreePostingsWriter extends BlockTreePostingsWriterBase { + + final static String CODEC = "PulsedPostingsTree"; + + // To add a new version, increment from the last one, and + // change VERSION_CURRENT to point to your new version: + final static int VERSION_START = 0; + + final static int VERSION_CURRENT = VERSION_START; + + private IndexOutput termsOut; + + private IndexOptions indexOptions; + private boolean storePayloads; + + private static class PendingTerm { + private final byte[] bytes; + public PendingTerm(byte[] bytes) { + this.bytes = bytes; + } + } + + private final List pendingTerms = new ArrayList(); + + // one entry per position + private final Position[] pending; + private int pendingCount = 0; // -1 once we've hit too many positions + private Position currentDoc; // first Position entry of current doc + + private static final class Position { + BytesRef payload; + int termFreq; // only incremented on first position for a given doc + int pos; + int docID; + } + + // TODO: -- lazy init this? ie, if every single term + // was inlined (eg for a "primary key" field) then we + // never need to use this fallback? Fallback writer for + // non-inlined terms: + final BlockTreePostingsWriterBase wrappedPostingsWriter; + + /** If the total number of positions (summed across all docs + * for this term) is <= maxPositions, then the postings are + * inlined into terms dict */ + public PulsingTreePostingsWriter(int maxPositions, BlockTreePostingsWriterBase wrappedPostingsWriter) throws IOException { + super(); + + pending = new Position[maxPositions]; + for(int i=0;i= the cutoff: + this.wrappedPostingsWriter = wrappedPostingsWriter; + } + + @Override + public void start(IndexOutput termsOut) throws IOException { + this.termsOut = termsOut; + CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT); + termsOut.writeVInt(pending.length); // encode maxPositions in header + wrappedPostingsWriter.start(termsOut); + } + + @Override + public void startTerm() { + //System.out.println("PW startTerm"); + assert pendingCount == 0; + } + + // TODO: -- should we NOT reuse across fields? would + // be cleaner + + // Currently, this instance is re-used across fields, so + // our parent calls setField whenever the field changes + @Override + public void setField(FieldInfo fieldInfo) { + this.indexOptions = fieldInfo.indexOptions; + //System.out.println("PW field=" + fieldInfo.name + " omitTF=" + omitTF); + storePayloads = fieldInfo.storePayloads; + wrappedPostingsWriter.setField(fieldInfo); + if (BlockTreeTermsWriter.DEBUG && fieldInfo.name.equals("id")) { + DEBUG = true; + } else { + DEBUG = false; + } + } + + private boolean DEBUG; + + @Override + public void startDoc(int docID, int termDocFreq) throws IOException { + assert docID >= 0: "got docID=" + docID; + + /* + if (termID != -1) { + if (docID == 0) { + baseDocID = termID; + } else if (baseDocID + docID != termID) { + throw new RuntimeException("WRITE: baseDocID=" + baseDocID + " docID=" + docID + " termID=" + termID); + } + } + */ + + //System.out.println("PW doc=" + docID); + if (DEBUG) { + System.out.println("PW docID=" + docID); + } + + if (pendingCount == pending.length) { + push(); + //System.out.println("PW: wrapped.finishDoc"); + wrappedPostingsWriter.finishDoc(); + } + + if (pendingCount != -1) { + assert pendingCount < pending.length; + currentDoc = pending[pendingCount]; + currentDoc.docID = docID; + if (indexOptions == IndexOptions.DOCS_ONLY) { + pendingCount++; + } else { + currentDoc.termFreq = termDocFreq; + } + } else { + // We've already seen too many docs for this term -- + // just forward to our fallback writer + wrappedPostingsWriter.startDoc(docID, termDocFreq); + } + } + + @Override + public void addPosition(int position, BytesRef payload) throws IOException { + + //System.out.println("PW pos=" + position + " payload=" + (payload == null ? "null" : payload.length + " bytes")); + if (pendingCount == pending.length) { + push(); + } + + if (pendingCount == -1) { + // We've already seen too many docs for this term -- + // just forward to our fallback writer + wrappedPostingsWriter.addPosition(position, payload); + } else { + // buffer up + final Position pos = pending[pendingCount++]; + pos.pos = position; + pos.docID = currentDoc.docID; + if (payload != null && payload.length > 0) { + if (pos.payload == null) { + pos.payload = new BytesRef(payload); + } else { + pos.payload.copy(payload); + } + } else if (pos.payload != null) { + pos.payload.length = 0; + } + } + } + + @Override + public void finishDoc() throws IOException { + //System.out.println("PW finishDoc"); + if (pendingCount == -1) { + wrappedPostingsWriter.finishDoc(); + } + } + + private final RAMOutputStream buffer = new RAMOutputStream(); + + private int baseDocID; + + /** Called when we are done adding docs to this term */ + @Override + public void finishTerm(TermStats stats) throws IOException { + //System.out.println("PW finishTerm docCount=" + stats.docFreq); + + assert pendingCount > 0 || pendingCount == -1; + + if (pendingCount == -1) { + wrappedPostingsWriter.finishTerm(stats); + // Must add null entry to record terms that our + // wrapped postings impl added + pendingTerms.add(null); + } else { + + // There were few enough total occurrences for this + // term, so we fully inline our postings data into + // terms dict, now: + + // TODO: it'd be better to share this encoding logic + // in some inner codec that knows how to write a + // single doc / single position, etc. This way if a + // given codec wants to store other interesting + // stuff, it could use this pulsing codec to do so + + if (indexOptions != IndexOptions.DOCS_ONLY) { + int lastDocID = 0; + int pendingIDX = 0; + while(pendingIDX < pendingCount) { + final Position doc = pending[pendingIDX]; + + final int delta = doc.docID - lastDocID; + lastDocID = doc.docID; + + //System.out.println(" write doc=" + doc.docID + " freq=" + doc.termFreq); + + if (doc.termFreq == 1) { + buffer.writeVInt((delta<<1)|1); + } else { + buffer.writeVInt(delta<<1); + buffer.writeVInt(doc.termFreq); + } + + int lastPos = 0; + int lastPayloadLength = -1; + for(int posIDX=0;posIDX files) throws IOException { + StandardTreePostingsReader.files(dir, segmentInfo, codecID, files); + BlockTreeTermsReader.files(dir, segmentInfo, codecID, files); + DefaultDocValuesConsumer.files(dir, segmentInfo, codecID, files, getDocValuesUseCFS()); + } + + @Override + public void getExtensions(Set extensions) { + getStandardExtensions(extensions); + DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS()); + } + + public static void getStandardExtensions(Set extensions) { + extensions.add(FREQ_EXTENSION); + extensions.add(PROX_EXTENSION); + BlockTreeTermsReader.getExtensions(extensions); + } + + @Override + public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException { + return new DefaultDocValuesConsumer(state, getDocValuesSortComparator(), getDocValuesUseCFS()); + } + + @Override + public PerDocValues docsProducer(SegmentReadState state) throws IOException { + return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context); + } +} Index: lucene/src/java/org/apache/lucene/index/codecs/standardtree/StandardTreePostingsReader.java --- /dev/null Thu Jan 01 00:00:00 1970 +0000 +++ lucene/src/java/org/apache/lucene/index/codecs/standardtree/StandardTreePostingsReader.java Mon Aug 01 12:09:21 2011 -0400 @@ -0,0 +1,888 @@ +package org.apache.lucene.index.codecs.standardtree; + +/** + * 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.Collection; + +import org.apache.lucene.index.DocsAndPositionsEnum; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.TermState; +import org.apache.lucene.index.codecs.BlockTreePostingsReaderBase; +import org.apache.lucene.index.codecs.BlockTreeTermState; +import org.apache.lucene.index.codecs.BlockTreeTermsWriter; +import org.apache.lucene.index.codecs.standard.DefaultSkipListReader; +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; + +/** Concrete class that reads the current doc/freq/skip + * postings format. + * @lucene.experimental */ + +public class StandardTreePostingsReader extends BlockTreePostingsReaderBase { + + private final IndexInput freqIn; + private final IndexInput proxIn; + //public static boolean DEBUG = BlockTreeTermsWriter.DEBUG; + + int skipInterval; + int maxSkipLevels; + int skipMinimum; + + //private String segment; + + public StandardTreePostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext ioContext, int codecId) throws IOException { + freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardTreeCodec.FREQ_EXTENSION), + ioContext); + //this.segment = segmentInfo.name; + if (segmentInfo.getHasProx()) { + boolean success = false; + try { + proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardTreeCodec.PROX_EXTENSION), + ioContext); + success = true; + } finally { + if (!success) { + freqIn.close(); + } + } + } else { + proxIn = null; + } + } + + public static void files(Directory dir, SegmentInfo segmentInfo, int codecID, Collection files) throws IOException { + files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardTreeCodec.FREQ_EXTENSION)); + if (segmentInfo.getHasProx()) { + files.add(IndexFileNames.segmentFileName(segmentInfo.name, codecID, StandardTreeCodec.PROX_EXTENSION)); + } + } + + @Override + public void init(IndexInput termsIn) throws IOException { + + // Make sure we are talking to the matching past writer + CodecUtil.checkHeader(termsIn, StandardTreePostingsWriter.CODEC, + StandardTreePostingsWriter.VERSION_START, StandardTreePostingsWriter.VERSION_START); + + skipInterval = termsIn.readInt(); + maxSkipLevels = termsIn.readInt(); + skipMinimum = termsIn.readInt(); + } + + // Must keep final because we do non-standard clone + private final static class StandardTermState extends BlockTreeTermState { + long freqOffset; + long proxOffset; + int skipOffset; + + // Only used by the "primary" TermState -- clones don't + // copy this (basically they are "transient"): + ByteArrayDataInput bytesReader; // TODO: should this NOT be in the TermState...? + byte[] bytes; + + @Override + public Object clone() { + StandardTermState other = new StandardTermState(); + other.copyFrom(this); + return other; + } + + @Override + public void copyFrom(TermState _other) { + super.copyFrom(_other); + StandardTermState other = (StandardTermState) _other; + freqOffset = other.freqOffset; + proxOffset = other.proxOffset; + skipOffset = other.skipOffset; + + // Do not copy bytes, bytesReader (else TermState is + // very heavy, ie drags around the entire block's + // byte[]). On seek back, if next() is in fact used + // (rare!), they will be re-read from disk. + } + + @Override + public String toString() { + return super.toString() + " freqFP=" + freqOffset + " proxFP=" + proxOffset + " skipOffset=" + skipOffset; + } + } + + @Override + public BlockTreeTermState newTermState() { + return new StandardTermState(); + } + + @Override + public void close() throws IOException { + try { + if (freqIn != null) { + freqIn.close(); + } + } finally { + if (proxIn != null) { + proxIn.close(); + } + } + } + + /* Reads but does not decode the byte[] blob holding + metadata for the current terms block */ + @Override + public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTreeTermState _termState) throws IOException { + final StandardTermState termState = (StandardTermState) _termState; + + final int len = termsIn.readVInt(); + + // nocommit: we should be able to do this?: + //if (len == 0) { + //return; + //} + + //if (DEBUG) System.out.println(" SPR.readTermsBlock bytes=" + len + " ts=" + _termState); + if (termState.bytes == null) { + termState.bytes = new byte[ArrayUtil.oversize(len, 1)]; + termState.bytesReader = new ByteArrayDataInput(); + } else if (termState.bytes.length < len) { + termState.bytes = new byte[ArrayUtil.oversize(len, 1)]; + } + + termsIn.readBytes(termState.bytes, 0, len); + termState.bytesReader.reset(termState.bytes, 0, len); + } + + @Override + public void resetTermsBlock(FieldInfo fieldInfo, BlockTreeTermState _termState) throws IOException { + //if (DEBUG) System.out.println(" SPR.resetTermsBlock ts=" + _termState); + final StandardTermState termState = (StandardTermState) _termState; + assert termState.bytes != null; + termState.bytesReader.rewind(); + } + + @Override + public void nextTerm(FieldInfo fieldInfo, BlockTreeTermState _termState) + throws IOException { + final StandardTermState termState = (StandardTermState) _termState; + //if (DEBUG) System.out.println(" stpr.nextTerm seg=" + segment + " tbOrd=" + termState.termBlockOrd + " bytesReader.fp=" + termState.bytesReader.getPosition()); + final boolean isFirstTerm = termState.termBlockOrd == 0; + + if (isFirstTerm) { + termState.freqOffset = termState.bytesReader.readVLong(); + } else { + termState.freqOffset += termState.bytesReader.readVLong(); + } + /* + if (DEBUG) { + System.out.println(" dF=" + termState.docFreq); + System.out.println(" freqFP=" + termState.freqOffset); + } + */ + assert termState.freqOffset < freqIn.length(); + + if (termState.docFreq >= skipMinimum) { + termState.skipOffset = termState.bytesReader.readVInt(); + //System.out.println(" skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length()); + assert termState.freqOffset + termState.skipOffset < freqIn.length(); + } else { + // undefined + } + + if (fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + if (isFirstTerm) { + termState.proxOffset = termState.bytesReader.readVLong(); + } else { + termState.proxOffset += termState.bytesReader.readVLong(); + } + //System.out.println(" proxFP=" + termState.proxOffset); + } + } + + @Override + public DocsEnum docs(FieldInfo fieldInfo, BlockTreeTermState termState, Bits liveDocs, DocsEnum reuse) throws IOException { + SegmentDocsEnum docsEnum; + if (reuse == null || !(reuse instanceof SegmentDocsEnum)) { + docsEnum = new SegmentDocsEnum(freqIn); + } else { + docsEnum = (SegmentDocsEnum) reuse; + if (docsEnum.startFreqIn != freqIn) { + // If you are using ParellelReader, and pass in a + // reused DocsEnum, it could have come from another + // reader also using standard codec + docsEnum = new SegmentDocsEnum(freqIn); + } + } + //System.out.println("SPR.docs ts=" + termState); + return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs); + } + + @Override + public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTreeTermState termState, Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException { + if (fieldInfo.indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + return null; + } + + // TODO: refactor + if (fieldInfo.storePayloads) { + SegmentDocsAndPositionsAndPayloadsEnum docsEnum; + if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsAndPayloadsEnum)) { + docsEnum = new SegmentDocsAndPositionsAndPayloadsEnum(freqIn, proxIn); + } else { + docsEnum = (SegmentDocsAndPositionsAndPayloadsEnum) reuse; + if (docsEnum.startFreqIn != freqIn) { + // If you are using ParellelReader, and pass in a + // reused DocsEnum, it could have come from another + // reader also using standard codec + docsEnum = new SegmentDocsAndPositionsAndPayloadsEnum(freqIn, proxIn); + } + } + return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs); + } else { + SegmentDocsAndPositionsEnum docsEnum; + if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsEnum)) { + docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn); + } else { + docsEnum = (SegmentDocsAndPositionsEnum) reuse; + if (docsEnum.startFreqIn != freqIn) { + // If you are using ParellelReader, and pass in a + // reused DocsEnum, it could have come from another + // reader also using standard codec + docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn); + } + } + return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs); + } + } + + // Decodes only docs + private class SegmentDocsEnum extends DocsEnum { + final IndexInput freqIn; + final IndexInput startFreqIn; + + boolean omitTF; // does current field omit term freq? + boolean storePayloads; // does current field store payloads? + + int limit; // number of docs in this posting + int ord; // how many docs we've read + int doc; // doc we last read + int freq; // freq we last read + + Bits liveDocs; + + long freqOffset; + int skipOffset; + + boolean skipped; + DefaultSkipListReader skipper; + + public SegmentDocsEnum(IndexInput freqIn) throws IOException { + startFreqIn = freqIn; + this.freqIn = (IndexInput) freqIn.clone(); + } + + public SegmentDocsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException { + omitTF = fieldInfo.indexOptions == IndexOptions.DOCS_ONLY; + if (omitTF) { + freq = 1; + } + storePayloads = fieldInfo.storePayloads; + this.liveDocs = liveDocs; + freqOffset = termState.freqOffset; + skipOffset = termState.skipOffset; + + // TODO: for full enum case (eg segment merging) this + // seek is unnecessary; maybe we can avoid in such + // cases + freqIn.seek(termState.freqOffset); + limit = termState.docFreq; + assert limit > 0; + ord = 0; + doc = 0; + //System.out.println(" sde limit=" + limit + " freqFP=" + freqOffset); + + skipped = false; + + return this; + } + + @Override + public int nextDoc() throws IOException { + //if (DEBUG) System.out.println(" stpr.nextDoc seg=" + segment + " fp=" + freqIn.getFilePointer()); + while(true) { + if (ord == limit) { + //if (DEBUG) System.out.println(" return doc=" + NO_MORE_DOCS); + return doc = NO_MORE_DOCS; + } + + ord++; + + // Decode next doc/freq pair + final int code = freqIn.readVInt(); + //System.out.println(" code=" + code); + if (omitTF) { + doc += code; + } else { + doc += code >>> 1; // shift off low bit + if ((code & 1) != 0) { // if low bit is set + freq = 1; // freq is one + } else { + freq = freqIn.readVInt(); // else read freq + } + } + + if (liveDocs == null || liveDocs.get(doc)) { + break; + } + } + + //if (DEBUG) System.out.println(" stpr.nextDoc return doc=" + doc); + return doc; + } + + @Override + public int read() throws IOException { + + final int[] docs = bulkResult.docs.ints; + final int[] freqs = bulkResult.freqs.ints; + int i = 0; + final int length = docs.length; + while (i < length && ord < limit) { + ord++; + // manually inlined call to next() for speed + final int code = freqIn.readVInt(); + if (omitTF) { + doc += code; + } else { + doc += code >>> 1; // shift off low bit + if ((code & 1) != 0) { // if low bit is set + freq = 1; // freq is one + } else { + freq = freqIn.readVInt(); // else read freq + } + } + + if (liveDocs == null || liveDocs.get(doc)) { + docs[i] = doc; + freqs[i] = freq; + ++i; + } + } + + return i; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int freq() { + return freq; + } + + @Override + public int advance(int target) throws IOException { + + if ((target - skipInterval) >= doc && limit >= skipMinimum) { + + // There are enough docs in the posting to have + // skip data, and it isn't too close. + + if (skipper == null) { + // This is the first time this enum has ever been used for skipping -- do lazy init + skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval); + } + + if (!skipped) { + + // This is the first time this posting has + // skipped since reset() was called, so now we + // load the skip data for this posting + + skipper.init(freqOffset + skipOffset, + freqOffset, 0, + limit, storePayloads); + + skipped = true; + } + + final int newOrd = skipper.skipTo(target); + + if (newOrd > ord) { + // Skipper moved + + ord = newOrd; + doc = skipper.getDoc(); + freqIn.seek(skipper.getFreqPointer()); + } + } + + // scan for the rest: + do { + nextDoc(); + } while (target > doc); + + return doc; + } + } + + // Decodes docs & positions. payloads are not present. + private class SegmentDocsAndPositionsEnum extends DocsAndPositionsEnum { + final IndexInput startFreqIn; + private final IndexInput freqIn; + private final IndexInput proxIn; + + int limit; // number of docs in this posting + int ord; // how many docs we've read + int doc; // doc we last read + int freq; // freq we last read + int position; + + Bits liveDocs; + + long freqOffset; + int skipOffset; + long proxOffset; + + int posPendingCount; + + boolean skipped; + DefaultSkipListReader skipper; + private long lazyProxPointer; + + public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException { + startFreqIn = freqIn; + this.freqIn = (IndexInput) freqIn.clone(); + this.proxIn = (IndexInput) proxIn.clone(); + } + + public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException { + assert fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS; + assert !fieldInfo.storePayloads; + + this.liveDocs = liveDocs; + + // TODO: for full enum case (eg segment merging) this + // seek is unnecessary; maybe we can avoid in such + // cases + freqIn.seek(termState.freqOffset); + lazyProxPointer = termState.proxOffset; + + limit = termState.docFreq; + assert limit > 0; + + ord = 0; + doc = 0; + position = 0; + + skipped = false; + posPendingCount = 0; + + freqOffset = termState.freqOffset; + proxOffset = termState.proxOffset; + skipOffset = termState.skipOffset; + //System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset); + + return this; + } + + @Override + public int nextDoc() throws IOException { + while(true) { + if (ord == limit) { + //System.out.println("StandardR.D&PE seg=" + segment + " nextDoc return doc=END"); + return doc = NO_MORE_DOCS; + } + + ord++; + + // Decode next doc/freq pair + final int code = freqIn.readVInt(); + + doc += code >>> 1; // shift off low bit + if ((code & 1) != 0) { // if low bit is set + freq = 1; // freq is one + } else { + freq = freqIn.readVInt(); // else read freq + } + posPendingCount += freq; + + if (liveDocs == null || liveDocs.get(doc)) { + break; + } + } + + position = 0; + + //System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc); + return doc; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int freq() { + return freq; + } + + @Override + public int advance(int target) throws IOException { + + //System.out.println("StandardR.D&PE advance target=" + target); + + if ((target - skipInterval) >= doc && limit >= skipMinimum) { + + // There are enough docs in the posting to have + // skip data, and it isn't too close + + if (skipper == null) { + // This is the first time this enum has ever been used for skipping -- do lazy init + skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval); + } + + if (!skipped) { + + // This is the first time this posting has + // skipped, since reset() was called, so now we + // load the skip data for this posting + + skipper.init(freqOffset+skipOffset, + freqOffset, proxOffset, + limit, false); + + skipped = true; + } + + final int newOrd = skipper.skipTo(target); + + if (newOrd > ord) { + // Skipper moved + ord = newOrd; + doc = skipper.getDoc(); + freqIn.seek(skipper.getFreqPointer()); + lazyProxPointer = skipper.getProxPointer(); + posPendingCount = 0; + position = 0; + } + } + + // Now, linear scan for the rest: + do { + nextDoc(); + } while (target > doc); + + return doc; + } + + @Override + public int nextPosition() throws IOException { + + if (lazyProxPointer != -1) { + proxIn.seek(lazyProxPointer); + lazyProxPointer = -1; + } + + // scan over any docs that were iterated without their positions + if (posPendingCount > freq) { + position = 0; + while(posPendingCount != freq) { + if ((proxIn.readByte() & 0x80) == 0) { + posPendingCount--; + } + } + } + + position += proxIn.readVInt(); + + posPendingCount--; + + assert posPendingCount >= 0: "nextPosition() was called too many times (more than freq() times) posPendingCount=" + posPendingCount; + + return position; + } + + /** Returns the payload at this position, or null if no + * payload was indexed. */ + @Override + public BytesRef getPayload() throws IOException { + throw new IOException("No payloads exist for this field!"); + } + + @Override + public boolean hasPayload() { + return false; + } + } + + // Decodes docs & positions & payloads + private class SegmentDocsAndPositionsAndPayloadsEnum extends DocsAndPositionsEnum { + final IndexInput startFreqIn; + private final IndexInput freqIn; + private final IndexInput proxIn; + + int limit; // number of docs in this posting + int ord; // how many docs we've read + int doc; // doc we last read + int freq; // freq we last read + int position; + + Bits liveDocs; + + long freqOffset; + int skipOffset; + long proxOffset; + + int posPendingCount; + int payloadLength; + boolean payloadPending; + + boolean skipped; + DefaultSkipListReader skipper; + private BytesRef payload; + private long lazyProxPointer; + + public SegmentDocsAndPositionsAndPayloadsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException { + startFreqIn = freqIn; + this.freqIn = (IndexInput) freqIn.clone(); + this.proxIn = (IndexInput) proxIn.clone(); + } + + public SegmentDocsAndPositionsAndPayloadsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException { + assert fieldInfo.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS; + assert fieldInfo.storePayloads; + if (payload == null) { + payload = new BytesRef(); + payload.bytes = new byte[1]; + } + + this.liveDocs = liveDocs; + + // TODO: for full enum case (eg segment merging) this + // seek is unnecessary; maybe we can avoid in such + // cases + freqIn.seek(termState.freqOffset); + lazyProxPointer = termState.proxOffset; + + limit = termState.docFreq; + ord = 0; + doc = 0; + position = 0; + + skipped = false; + posPendingCount = 0; + payloadPending = false; + + freqOffset = termState.freqOffset; + proxOffset = termState.proxOffset; + skipOffset = termState.skipOffset; + //System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset + " this=" + this); + + return this; + } + + @Override + public int nextDoc() throws IOException { + while(true) { + if (ord == limit) { + //System.out.println("StandardR.D&PE seg=" + segment + " nextDoc return doc=END"); + return doc = NO_MORE_DOCS; + } + + ord++; + + // Decode next doc/freq pair + final int code = freqIn.readVInt(); + + doc += code >>> 1; // shift off low bit + if ((code & 1) != 0) { // if low bit is set + freq = 1; // freq is one + } else { + freq = freqIn.readVInt(); // else read freq + } + posPendingCount += freq; + + if (liveDocs == null || liveDocs.get(doc)) { + break; + } + } + + position = 0; + + //System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc); + return doc; + } + + @Override + public int docID() { + return doc; + } + + @Override + public int freq() { + return freq; + } + + @Override + public int advance(int target) throws IOException { + + //System.out.println("StandardR.D&PE advance seg=" + segment + " target=" + target + " this=" + this); + + if ((target - skipInterval) >= doc && limit >= skipMinimum) { + + // There are enough docs in the posting to have + // skip data, and it isn't too close + + if (skipper == null) { + // This is the first time this enum has ever been used for skipping -- do lazy init + skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval); + } + + if (!skipped) { + + // This is the first time this posting has + // skipped, since reset() was called, so now we + // load the skip data for this posting + //System.out.println(" init skipper freqOffset=" + freqOffset + " skipOffset=" + skipOffset + " vs len=" + freqIn.length()); + skipper.init(freqOffset+skipOffset, + freqOffset, proxOffset, + limit, true); + + skipped = true; + } + + final int newOrd = skipper.skipTo(target); + + if (newOrd > ord) { + // Skipper moved + ord = newOrd; + doc = skipper.getDoc(); + freqIn.seek(skipper.getFreqPointer()); + lazyProxPointer = skipper.getProxPointer(); + posPendingCount = 0; + position = 0; + payloadPending = false; + payloadLength = skipper.getPayloadLength(); + } + } + + // Now, linear scan for the rest: + do { + nextDoc(); + } while (target > doc); + + return doc; + } + + @Override + public int nextPosition() throws IOException { + + if (lazyProxPointer != -1) { + proxIn.seek(lazyProxPointer); + lazyProxPointer = -1; + } + + if (payloadPending && payloadLength > 0) { + // payload of last position as never retrieved -- skip it + proxIn.seek(proxIn.getFilePointer() + payloadLength); + payloadPending = false; + } + + // scan over any docs that were iterated without their positions + while(posPendingCount > freq) { + + final int code = proxIn.readVInt(); + + if ((code & 1) != 0) { + // new payload length + payloadLength = proxIn.readVInt(); + assert payloadLength >= 0; + } + + assert payloadLength != -1; + proxIn.seek(proxIn.getFilePointer() + payloadLength); + + posPendingCount--; + position = 0; + payloadPending = false; + //System.out.println("StandardR.D&PE skipPos"); + } + + // read next position + if (payloadPending && payloadLength > 0) { + // payload wasn't retrieved for last position + proxIn.seek(proxIn.getFilePointer()+payloadLength); + } + + final int code = proxIn.readVInt(); + if ((code & 1) != 0) { + // new payload length + payloadLength = proxIn.readVInt(); + assert payloadLength >= 0; + } + assert payloadLength != -1; + + payloadPending = true; + position += code >>> 1; + + posPendingCount--; + + assert posPendingCount >= 0: "nextPosition() was called too many times (more than freq() times) posPendingCount=" + posPendingCount; + + //System.out.println("StandardR.D&PE nextPos return pos=" + position); + return position; + } + + /** Returns the payload at this position, or null if no + * payload was indexed. */ + @Override + public BytesRef getPayload() throws IOException { + assert lazyProxPointer == -1; + assert posPendingCount < freq; + if (!payloadPending) { + throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once."); + } + if (payloadLength > payload.bytes.length) { + payload.grow(payloadLength); + } + + proxIn.readBytes(payload.bytes, 0, payloadLength); + payload.length = payloadLength; + payloadPending = false; + + return payload; + } + + @Override + public boolean hasPayload() { + return payloadPending && payloadLength > 0; + } + } +} Index: lucene/src/java/org/apache/lucene/index/codecs/standardtree/StandardTreePostingsWriter.java --- /dev/null Thu Jan 01 00:00:00 1970 +0000 +++ lucene/src/java/org/apache/lucene/index/codecs/standardtree/StandardTreePostingsWriter.java Mon Aug 01 12:09:21 2011 -0400 @@ -0,0 +1,322 @@ +package org.apache.lucene.index.codecs.standardtree; + +/** + * 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. + */ + +/** Consumes doc & freq, writing them using the current + * index file format */ + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.DocsEnum; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfo.IndexOptions; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.codecs.BlockTreePostingsWriterBase; +import org.apache.lucene.index.codecs.BlockTreeTermsWriter; +import org.apache.lucene.index.codecs.TermStats; +import org.apache.lucene.index.codecs.standard.DefaultSkipListWriter; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RAMOutputStream; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CodecUtil; + +/** @lucene.experimental */ +public final class StandardTreePostingsWriter extends BlockTreePostingsWriterBase { + final static String CODEC = "StandardTreePostingsWriterImpl"; + + public static boolean DEBUG = BlockTreeTermsWriter.DEBUG; + + // Increment version to change it: + final static int VERSION_START = 0; + final static int VERSION_CURRENT = VERSION_START; + + final IndexOutput freqOut; + final IndexOutput proxOut; + final DefaultSkipListWriter skipListWriter; + /** Expert: The fraction of TermDocs entries stored in skip tables, + * used to accelerate {@link DocsEnum#advance(int)}. Larger values result in + * smaller indexes, greater acceleration, but fewer accelerable cases, while + * smaller values result in bigger indexes, less acceleration and more + * accelerable cases. More detailed experiments would be useful here. */ + final int skipInterval = 16; + + /** + * Expert: minimum docFreq to write any skip data at all + */ + final int skipMinimum = skipInterval; + + /** Expert: The maximum number of skip levels. Smaller values result in + * slightly smaller indexes, but slower skipping in big posting lists. + */ + final int maxSkipLevels = 10; + final int totalNumDocs; + IndexOutput termsOut; + + IndexOptions indexOptions; + boolean storePayloads; + // Starts a new term + long freqStart; + long proxStart; + FieldInfo fieldInfo; + int lastPayloadLength; + int lastPosition; + + //private String segment; + + public StandardTreePostingsWriter(SegmentWriteState state) throws IOException { + super(); + //this.segment = state.segmentName; + String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardTreeCodec.FREQ_EXTENSION); + freqOut = state.directory.createOutput(fileName, state.context); + if (state.fieldInfos.hasProx()) { + // At least one field does not omit TF, so create the + // prox file + fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardTreeCodec.PROX_EXTENSION); + proxOut = state.directory.createOutput(fileName, state.context); + } else { + // Every field omits TF so we will write no prox file + proxOut = null; + } + + totalNumDocs = state.numDocs; + + skipListWriter = new DefaultSkipListWriter(skipInterval, + maxSkipLevels, + state.numDocs, + freqOut, + proxOut); + } + + @Override + public void start(IndexOutput termsOut) throws IOException { + this.termsOut = termsOut; + CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT); + termsOut.writeInt(skipInterval); // write skipInterval + termsOut.writeInt(maxSkipLevels); // write maxSkipLevels + termsOut.writeInt(skipMinimum); // write skipMinimum + } + + @Override + public void startTerm() { + freqStart = freqOut.getFilePointer(); + if (proxOut != null) { + proxStart = proxOut.getFilePointer(); + // force first payload to write its length + lastPayloadLength = -1; + } + skipListWriter.resetSkip(); + } + + // Currently, this instance is re-used across fields, so + // our parent calls setField whenever the field changes + @Override + public void setField(FieldInfo fieldInfo) { + //System.out.println("SPW: setField"); + /* + if (BlockTreeTermsWriter.DEBUG && fieldInfo.name.equals("id")) { + DEBUG = true; + } else { + DEBUG = false; + } + */ + this.fieldInfo = fieldInfo; + indexOptions = fieldInfo.indexOptions; + storePayloads = fieldInfo.storePayloads; + //System.out.println(" set init blockFreqStart=" + freqStart); + //System.out.println(" set init blockProxStart=" + proxStart); + } + + int lastDocID; + int df; + + /** Adds a new doc in this term. If this returns null + * then we just skip consuming positions/payloads. */ + @Override + public void startDoc(int docID, int termDocFreq) throws IOException { + //System.out.println("StandardW: startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq); + if (DEBUG) System.out.println("SPW.startDoc docID=" + docID + " freqOut.fp=" + freqOut.getFilePointer()); + + final int delta = docID - lastDocID; + + if (docID < 0 || (df > 0 && delta <= 0)) { + throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )"); + } + + if ((++df % skipInterval) == 0) { + skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength); + skipListWriter.bufferSkip(df); + } + + assert docID < totalNumDocs: "docID=" + docID + " totalNumDocs=" + totalNumDocs; + + lastDocID = docID; + if (indexOptions == IndexOptions.DOCS_ONLY) { + freqOut.writeVInt(delta); + } else if (1 == termDocFreq) { + freqOut.writeVInt((delta<<1) | 1); + } else { + freqOut.writeVInt(delta<<1); + freqOut.writeVInt(termDocFreq); + } + + lastPosition = 0; + } + + /** Add a new position & payload */ + @Override + public void addPosition(int position, BytesRef payload) throws IOException { + //System.out.println("StandardW: addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer()); + assert indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS: "invalid indexOptions: " + indexOptions; + assert proxOut != null; + + final int delta = position - lastPosition; + + assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition; // not quite right (if pos=0 is repeated twice we don't catch it) + + lastPosition = position; + + if (storePayloads) { + final int payloadLength = payload == null ? 0 : payload.length; + + if (payloadLength != lastPayloadLength) { + lastPayloadLength = payloadLength; + proxOut.writeVInt((delta<<1)|1); + proxOut.writeVInt(payloadLength); + } else { + proxOut.writeVInt(delta << 1); + } + + if (payloadLength > 0) { + proxOut.writeBytes(payload.bytes, payload.offset, payloadLength); + } + } else { + proxOut.writeVInt(delta); + } + } + + @Override + public void finishDoc() { + } + + private static class PendingTerm { + public final long freqStart; + public final long proxStart; + public final int skipOffset; + + public PendingTerm(long freqStart, long proxStart, int skipOffset) { + this.freqStart = freqStart; + this.proxStart = proxStart; + this.skipOffset = skipOffset; + } + } + + private final List pendingTerms = new ArrayList(); + + /** Called when we are done adding docs to this term */ + @Override + public void finishTerm(TermStats stats) throws IOException { + + //System.out.println("StandardW.finishTerm seg=" + segment); + assert stats.docFreq > 0; + + // TODO: wasteful we are counting this (counting # docs + // for this term) in two places? + assert stats.docFreq == df; + + final int skipOffset; + if (df >= skipMinimum) { + skipOffset = (int) (skipListWriter.writeSkip(freqOut)-freqStart); + } else { + skipOffset = -1; + } + + pendingTerms.add(new PendingTerm(freqStart, proxStart, skipOffset)); + + lastDocID = 0; + df = 0; + } + + private final RAMOutputStream bytesWriter = new RAMOutputStream(); + + @Override + public void flushTermsBlock(int start, int count) throws IOException { + if (DEBUG) System.out.println("SPW.flushTermsBlock start=" + start + " count=" + count + " left=" + (pendingTerms.size()-count)); + + if (count == 0) { + // nocommit: silly? can we avoid this if we know + // block has no terms? + termsOut.writeByte((byte) 0); + return; + } + + assert start <= pendingTerms.size(); + assert count <= start; + + final int limit = pendingTerms.size() - start + count; + final PendingTerm firstTerm = pendingTerms.get(limit - count); + // First term in block is abs coded: + bytesWriter.writeVLong(firstTerm.freqStart); + + if (firstTerm.skipOffset != -1) { + assert firstTerm.skipOffset > 0; + bytesWriter.writeVInt(firstTerm.skipOffset); + } + if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + bytesWriter.writeVLong(firstTerm.proxStart); + } + long lastFreqStart = firstTerm.freqStart; + long lastProxStart = firstTerm.proxStart; + for(int idx=limit-count+1; idx 0; + bytesWriter.writeVInt(term.skipOffset); + } + if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) { + bytesWriter.writeVLong(term.proxStart - lastProxStart); + lastProxStart = term.proxStart; + } + } + + termsOut.writeVInt((int) bytesWriter.getFilePointer()); + bytesWriter.writeTo(termsOut); + bytesWriter.reset(); + + // Remove the terms we just wrote: + pendingTerms.subList(limit-count, limit).clear(); + } + + @Override + public void close() throws IOException { + try { + freqOut.close(); + } finally { + if (proxOut != null) { + proxOut.close(); + } + } + } +} Index: lucene/src/java/org/apache/lucene/search/AutomatonQuery.java --- lucene/src/java/org/apache/lucene/search/AutomatonQuery.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/search/AutomatonQuery.java Mon Aug 01 12:09:21 2011 -0400 @@ -22,12 +22,12 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton; import org.apache.lucene.util.ToStringUtils; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.BasicAutomata; import org.apache.lucene.util.automaton.BasicOperations; +import org.apache.lucene.util.automaton.CompiledAutomaton; import org.apache.lucene.util.automaton.MinimizationOperations; import org.apache.lucene.util.automaton.SpecialOperations; @@ -130,12 +130,12 @@ } }; } else { - final AutomatonTermsEnum.CompiledAutomaton compiled = + final CompiledAutomaton compiled = new CompiledAutomaton(automaton, SpecialOperations.isFinite(automaton)); factory = new TermsEnumFactory() { @Override protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException { - return new AutomatonTermsEnum(terms.iterator(), compiled); + return terms.intersect(compiled, null); } }; } Index: lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java --- lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java Mon Aug 01 12:07:14 2011 -0400 +++ /dev/null Thu Jan 01 00:00:00 1970 +0000 @@ -1,343 +0,0 @@ -package org.apache.lucene.search; - -/** - * 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.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.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: - *

    - *
  1. As long as matches are successful, keep reading sequentially. - *
  2. 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 { - 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 */ - } - - /** - * immutable class with everything this enum needs. - */ - public static class CompiledAutomaton { - public final ByteRunAutomaton runAutomaton; - public final Transition[][] sortedTransitions; - public final BytesRef commonSuffixRef; - public final boolean finite; - - public CompiledAutomaton(Automaton automaton, boolean finite) { - Automaton utf8 = new UTF32ToUTF8().convert(automaton); - runAutomaton = new ByteRunAutomaton(utf8, true); - sortedTransitions = utf8.getSortedTransitions(); - this.finite = finite; - if (finite) { - commonSuffixRef = null; - } else { - commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8); - } - } - } -} Index: lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java --- lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java Mon Aug 01 12:09:21 2011 -0400 @@ -42,7 +42,7 @@ public abstract class FilteredTermsEnum extends TermsEnum { private BytesRef initialSeekTerm = null; - private boolean doSeek = true; + private boolean doSeek; private BytesRef actualTerm = null; private final TermsEnum tenum; @@ -64,8 +64,17 @@ * @param tenum the terms enumeration to filter. */ public FilteredTermsEnum(final TermsEnum tenum) { + this(tenum, true); + } + + /** + * Creates a filtered {@link TermsEnum} on a terms enum. + * @param tenum the terms enumeration to filter. + */ + public FilteredTermsEnum(final TermsEnum tenum, final boolean startWithSeek) { assert tenum != null; this.tenum = tenum; + doSeek = startWithSeek; } /** @@ -190,18 +199,23 @@ @SuppressWarnings("fallthrough") @Override public BytesRef next() throws IOException { + //System.out.println("FTE.next doSeek=" + doSeek); + //new Throwable().printStackTrace(System.out); for (;;) { // Seek or forward the iterator if (doSeek) { doSeek = false; final BytesRef t = nextSeekTerm(actualTerm); + //System.out.println(" seek to t=" + (t == null ? "null" : t.utf8ToString()) + " tenum=" + tenum); // Make sure we always seek forward: assert actualTerm == null || t == null || getComparator().compare(t, actualTerm) > 0: "curTerm=" + actualTerm + " seekTerm=" + t; if (t == null || tenum.seekCeil(t, false) == SeekStatus.END) { // no more terms to seek to or enum exhausted + //System.out.println(" return null"); return null; } actualTerm = tenum.term(); + //System.out.println(" got term=" + actualTerm.utf8ToString()); } else { actualTerm = tenum.next(); if (actualTerm == null) { Index: lucene/src/java/org/apache/lucene/search/FuzzyQuery.java --- lucene/src/java/org/apache/lucene/search/FuzzyQuery.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/search/FuzzyQuery.java Mon Aug 01 12:09:21 2011 -0400 @@ -137,12 +137,10 @@ @Override protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException { - TermsEnum tenum = terms.iterator(); - if (!termLongEnough) { // can only match if it's exact - return new SingleTermsEnum(tenum, term); + return new SingleTermsEnum(terms.iterator(), term); } - return new FuzzyTermsEnum(tenum, atts, getTerm(), minimumSimilarity, prefixLength); + return new FuzzyTermsEnum(terms, atts, getTerm(), minimumSimilarity, prefixLength); } /** Index: lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java --- lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Mon Aug 01 12:09:21 2011 -0400 @@ -17,12 +17,17 @@ * limitations under the License. */ +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; + import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermState; +import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.search.AutomatonTermsEnum.CompiledAutomaton; import org.apache.lucene.util.Attribute; import org.apache.lucene.util.AttributeImpl; import org.apache.lucene.util.AttributeSource; @@ -34,13 +39,9 @@ import org.apache.lucene.util.automaton.BasicAutomata; import org.apache.lucene.util.automaton.BasicOperations; import org.apache.lucene.util.automaton.ByteRunAutomaton; +import org.apache.lucene.util.automaton.CompiledAutomaton; import org.apache.lucene.util.automaton.LevenshteinAutomata; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.List; - /** Subclass of TermsEnum for enumerating all terms that are similar * to the specified filter term. * @@ -72,7 +73,7 @@ private int maxEdits; private final boolean raw; - private final TermsEnum tenum; + private final Terms terms; private final Term term; private final int termText[]; private final int realPrefixLength; @@ -94,7 +95,7 @@ * @param prefixLength Length of required common prefix. Default value is 0. * @throws IOException */ - public FuzzyTermsEnum(TermsEnum tenum, AttributeSource atts, Term term, + public FuzzyTermsEnum(Terms terms, AttributeSource atts, Term term, final float minSimilarity, final int prefixLength) throws IOException { if (minSimilarity >= 1.0f && minSimilarity != (int)minSimilarity) throw new IllegalArgumentException("fractional edit distances are not allowed"); @@ -102,7 +103,7 @@ throw new IllegalArgumentException("minimumSimilarity cannot be less than 0"); if(prefixLength < 0) throw new IllegalArgumentException("prefixLength cannot be less than 0"); - this.tenum = tenum; + this.terms = terms; this.term = term; // convert the string into a utf32 int[] representation for fast comparisons @@ -142,9 +143,10 @@ private TermsEnum getAutomatonEnum(int editDistance, BytesRef lastTerm) throws IOException { final List runAutomata = initAutomata(editDistance); + System.out.println("GET AE ed=" + editDistance); if (editDistance < runAutomata.size()) { - return new AutomatonFuzzyTermsEnum(runAutomata.subList(0, editDistance + 1) - .toArray(new CompiledAutomaton[editDistance + 1]), lastTerm); + return new AutomatonFuzzyTermsEnum(terms.intersect(runAutomata.get(runAutomata.size()-1), lastTerm), + runAutomata.subList(0, editDistance + 1).toArray(new CompiledAutomaton[editDistance + 1])); } else { return null; } @@ -152,6 +154,7 @@ /** initialize levenshtein DFAs up to maxDistance, if possible */ private List initAutomata(int maxDistance) { + // nocommit -- is this called multiple times per query!? final List runAutomata = dfaAtt.automata(); if (runAutomata.size() <= maxDistance && maxDistance <= LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) { @@ -301,36 +304,40 @@ public BytesRef term() throws IOException { return actualEnum.term(); } - + /** - * Implement fuzzy enumeration with automaton. + * Implement fuzzy enumeration with Terms.intersect. *

* This is the fastest method as opposed to LinearFuzzyTermsEnum: * as enumeration is logarithmic to the number of terms (instead of linear) * and comparison is linear to length of the term (rather than quadratic) */ - private class AutomatonFuzzyTermsEnum extends AutomatonTermsEnum { + private class AutomatonFuzzyTermsEnum extends FilteredTermsEnum { private final ByteRunAutomaton matchers[]; private final BytesRef termRef; - private final BytesRef lastTerm; private final BoostAttribute boostAtt = attributes().addAttribute(BoostAttribute.class); - public AutomatonFuzzyTermsEnum(CompiledAutomaton compiled[], - BytesRef lastTerm) throws IOException { - super(tenum, compiled[compiled.length - 1]); + public AutomatonFuzzyTermsEnum(TermsEnum tenum, CompiledAutomaton compiled[]) + throws IOException { + super(tenum, false); this.matchers = new ByteRunAutomaton[compiled.length]; for (int i = 0; i < compiled.length; i++) this.matchers[i] = compiled[i].runAutomaton; - this.lastTerm = lastTerm; termRef = new BytesRef(term.text()); } - + + // nocommit -- we lost the more efficient (always seek) + // accept for FuzzyTermsEnum in Terms.intersect! so on + // non-intersect-capable terms dicts this is slower? + // maybe we stuff a hint (boolean alwaysSeek) into CA? + /** finds the smallest Lev(n) DFA that accepts the term. */ @Override protected AcceptStatus accept(BytesRef term) { + //System.out.println("AFTE.accept term=" + term); int ed = matchers.length - 1; if (matches(term, ed)) { // we match the outer dfa @@ -342,24 +349,27 @@ break; } } + //System.out.println("CHECK term=" + term.utf8ToString() + " ed=" + ed); // scale to a boost and return (if similarity > minSimilarity) if (ed == 0) { // exact match boostAtt.setBoost(1.0F); - return AcceptStatus.YES_AND_SEEK; + //System.out.println(" yes"); + return AcceptStatus.YES; } else { final int codePointCount = UnicodeUtil.codePointCount(term); final float similarity = 1.0f - ((float) ed / (float) - (Math.min(codePointCount, termLength))); + (Math.min(codePointCount, termLength))); if (similarity > minSimilarity) { boostAtt.setBoost((similarity - minSimilarity) * scale_factor); - return AcceptStatus.YES_AND_SEEK; + //System.out.println(" yes"); + return AcceptStatus.YES; } else { - return AcceptStatus.NO_AND_SEEK; + return AcceptStatus.NO; } } } else { - return AcceptStatus.NO_AND_SEEK; + return AcceptStatus.NO; } } @@ -367,16 +377,8 @@ final boolean matches(BytesRef term, int k) { return k == 0 ? term.equals(termRef) : matchers[k].run(term.bytes, term.offset, term.length); } - - /** defers to superclass, except can start at an arbitrary location */ - @Override - protected BytesRef nextSeekTerm(BytesRef term) throws IOException { - if (term == null) - term = lastTerm; - return super.nextSeekTerm(term); - } } - + /** * Implement fuzzy enumeration with linear brute force. */ @@ -408,7 +410,7 @@ * @throws IOException */ public LinearFuzzyTermsEnum() throws IOException { - super(tenum); + super(terms.iterator()); this.text = new int[termLength - realPrefixLength]; System.arraycopy(termText, realPrefixLength, text, 0, text.length); Index: lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java --- lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/search/TermCollectingRewrite.java Mon Aug 01 12:09:21 2011 -0400 @@ -21,16 +21,17 @@ import java.util.Comparator; import org.apache.lucene.index.Fields; +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.IndexReader.ReaderContext; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.index.IndexReader.AtomicReaderContext; -import org.apache.lucene.index.IndexReader.ReaderContext; +import org.apache.lucene.index.codecs.BlockTreeTermsWriter; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.ReaderUtil; import org.apache.lucene.util.TermContext; -import org.apache.lucene.util.ReaderUtil; abstract class TermCollectingRewrite extends MultiTermQuery.RewriteMethod { @@ -51,6 +52,9 @@ Comparator lastTermComp = null; final AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext); for (AtomicReaderContext context : leaves) { + if (BlockTreeTermsWriter.DEBUG) { + System.out.println("\nTERM COLLECTING REWRITE: now switch to seg=" + context.reader); + } final Fields fields = context.reader.fields(); if (fields == null) { // reader has no fields @@ -81,6 +85,8 @@ if (!collector.collect(bytes)) return; // interrupt whole term collection, so also don't iterate other subReaders } + + //System.out.println("terms reader=" + context.reader + " stats: " + termsEnum.getStats()); } } Index: lucene/src/java/org/apache/lucene/search/TermQuery.java --- lucene/src/java/org/apache/lucene/search/TermQuery.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/search/TermQuery.java Mon Aug 01 12:09:21 2011 -0400 @@ -98,17 +98,18 @@ TermsEnum getTermsEnum(AtomicReaderContext context) throws IOException { final TermState state = termStates.get(context.ord); if (state == null) { // term is not present in that reader - assert termNotInReader(context.reader, term.field(), term.bytes()) : "no termstate found but term exists in reader"; + assert termNotInReader(context.reader, term.field(), term.bytes()) : "no termstate found but term exists in reader term=" + term; return null; } - final TermsEnum termsEnum = context.reader.terms(term.field()) - .getThreadTermsEnum(); + //System.out.println("LD=" + reader.getLiveDocs() + " set?=" + (reader.getLiveDocs() != null ? reader.getLiveDocs().get(0) : "null")); + final TermsEnum termsEnum = context.reader.terms(term.field()).getThreadTermsEnum(); termsEnum.seekExact(term.bytes(), state); return termsEnum; } private boolean termNotInReader(IndexReader reader, String field, BytesRef bytes) throws IOException { // only called from assert + //System.out.println("TQ.termNotInReader reader=" + reader + " term=" + field + ":" + bytes.utf8ToString()); final Terms terms = reader.terms(field); return terms == null || terms.docFreq(bytes) == 0; } Index: lucene/src/java/org/apache/lucene/search/TermScorer.java --- lucene/src/java/org/apache/lucene/search/TermScorer.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/search/TermScorer.java Mon Aug 01 12:09:21 2011 -0400 @@ -70,6 +70,7 @@ public boolean score(Collector c, int end, int firstDocID) throws IOException { c.setScorer(this); while (doc < end) { // for docs in window + //System.out.println("TS: collect doc=" + doc); c.collect(doc); // collect score if (++pointer >= pointerMax) { refillBuffer(); Index: lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java --- lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/search/TopTermsRewrite.java Mon Aug 01 12:09:21 2011 -0400 @@ -84,6 +84,8 @@ boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class); } + // nocommit -- can we assert we never double-add term into PQ + @Override public boolean collect(BytesRef bytes) throws IOException { final float boost = boostAtt.getBoost(); Index: lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java --- lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/store/ByteArrayDataInput.java Mon Aug 01 12:09:21 2011 -0400 @@ -43,16 +43,30 @@ reset(bytes, 0, bytes.length); } + public void rewind() { + // nocommit -- not right if .reset was called w/ + // non-zero offset... + pos = 0; + } + public int getPosition() { return pos; } + public void setPosition(int pos) { + this.pos = pos; + } + public void reset(byte[] bytes, int offset, int len) { this.bytes = bytes; pos = offset; limit = offset + len; } + public int length() { + return limit; + } + public boolean eof() { return pos == limit; } Index: lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java --- lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java Mon Aug 01 12:09:21 2011 -0400 @@ -189,14 +189,14 @@ } @Override - public synchronized IndexInput openInput(String id, IOContext context) throws IOException { + public synchronized IndexInput openInput(String fileName, IOContext context) throws IOException { ensureOpen(); assert !openForWrite; - id = IndexFileNames.stripSegmentName(id); + final String id = IndexFileNames.stripSegmentName(fileName); final FileEntry entry = entries.get(id); - if (entry == null) - throw new IOException("No sub-file with id " + id + " found (files: " + entries.keySet() + ")"); - + if (entry == null) { + throw new IOException("No sub-file with id " + id + " found (fileName=" + fileName + " files: " + entries.keySet() + ")"); + } return openInputSlice(id, entry.offset, entry.length, readBufferSize); } Index: lucene/src/java/org/apache/lucene/util/BytesRef.java --- lucene/src/java/org/apache/lucene/util/BytesRef.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/util/BytesRef.java Mon Aug 01 12:09:21 2011 -0400 @@ -65,6 +65,18 @@ this.bytes = new byte[capacity]; } + /** Incoming IntsRef values must be Byte.MIN_VALUE - + * Byte.MAX_VALUE. */ + public BytesRef(IntsRef intsRef) { + bytes = new byte[intsRef.length]; + for(int idx=0;idx= Byte.MIN_VALUE && v <= Byte.MAX_VALUE; + bytes[idx] = (byte) v; + } + length = intsRef.length; + } + /** * @param text Initialize the byte[] from the UTF8 bytes * for the provided Sring. This must be well-formed Index: lucene/src/java/org/apache/lucene/util/TermContext.java --- lucene/src/java/org/apache/lucene/util/TermContext.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/util/TermContext.java Mon Aug 01 12:09:21 2011 -0400 @@ -21,14 +21,15 @@ import java.util.Arrays; import org.apache.lucene.index.Fields; +import org.apache.lucene.index.IndexReader.AtomicReaderContext; +import org.apache.lucene.index.IndexReader.ReaderContext; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; 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.IndexReader.AtomicReaderContext; -import org.apache.lucene.index.IndexReader.ReaderContext; -import org.apache.lucene.index.TermsEnum.SeekStatus; +import org.apache.lucene.index.codecs.BlockTreeTermsWriter; /** * Maintains a {@link IndexReader} {@link TermState} view over @@ -45,6 +46,9 @@ private int docFreq; private long totalTermFreq; + public static boolean DEBUG = BlockTreeTermsWriter.DEBUG; + + /** * Creates an empty {@link TermContext} from a {@link ReaderContext} */ @@ -85,7 +89,9 @@ final BytesRef bytes = term.bytes(); final TermContext perReaderTermState = new TermContext(context); final AtomicReaderContext[] leaves = ReaderUtil.leaves(context); + if (DEBUG) System.out.println("prts.build term=" + term); for (int i = 0; i < leaves.length; i++) { + if (DEBUG) System.out.println(" r=" + leaves[i].reader); final Fields fields = leaves[i].reader.fields(); if (fields != null) { final Terms terms = fields.terms(field); @@ -93,6 +99,7 @@ final TermsEnum termsEnum = terms.getThreadTermsEnum(); // thread-private don't share! if (termsEnum.seekExact(bytes, cache)) { final TermState termState = termsEnum.termState(); + if (DEBUG) System.out.println(" found"); perReaderTermState.register(termState, leaves[i].ord, termsEnum.docFreq(), termsEnum.totalTermFreq()); } } Index: lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java --- /dev/null Thu Jan 01 00:00:00 1970 +0000 +++ lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java Mon Aug 01 12:09:21 2011 -0400 @@ -0,0 +1,44 @@ +package org.apache.lucene.util.automaton; + +/** + * 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.util.BytesRef; + +/** + * immutable class with everything this enum needs. + * + * @lucene.experimental + */ +public class CompiledAutomaton { + public final ByteRunAutomaton runAutomaton; + public final Transition[][] sortedTransitions; + public final BytesRef commonSuffixRef; + public final boolean finite; + + public CompiledAutomaton(Automaton automaton, boolean finite) { + Automaton utf8 = new UTF32ToUTF8().convert(automaton); + runAutomaton = new ByteRunAutomaton(utf8, true); + sortedTransitions = utf8.getSortedTransitions(); + this.finite = finite; + if (finite) { + commonSuffixRef = null; + } else { + commonSuffixRef = SpecialOperations.getCommonSuffixBytesRef(utf8); + } + } +} Index: lucene/src/java/org/apache/lucene/util/fst/Builder.java --- lucene/src/java/org/apache/lucene/util/fst/Builder.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/util/fst/Builder.java Mon Aug 01 12:09:21 2011 -0400 @@ -23,7 +23,12 @@ import org.apache.lucene.util.IntsRef; import org.apache.lucene.util.fst.FST.INPUT_TYPE; // javadoc +// nocommit +import org.apache.lucene.index.codecs.BlockTreeTermsWriter; + import java.io.IOException; +import java.util.List; +import java.util.ArrayList; /** * Builds a compact FST (maps an IntsRef term to an arbitrary @@ -53,6 +58,8 @@ private final FST fst; private final T NO_OUTPUT; + public boolean DEBUG = BlockTreeTermsWriter.DEBUG; + // simplistic pruning: we prune node (and all following // nodes) if less than this number of terms go through it: private final int minSuffixCount1; @@ -73,13 +80,21 @@ // current "frontier" private UnCompiledNode[] frontier; + // Expert: you pass an instance of this if you want to do + // something "custom" as suffixes are "frozen": + public static abstract class FreezeTail { + public abstract void freeze(final UnCompiledNode[] frontier, int prefixLenPlus1, IntsRef prevInput) throws IOException; + } + + private final FreezeTail freezeTail; + /** * Instantiates an FST/FSA builder without any pruning. A shortcut * to {@link #Builder(FST.INPUT_TYPE, int, int, boolean, boolean, int, Outputs)} with * pruning options turned off. */ public Builder(FST.INPUT_TYPE inputType, Outputs outputs) { - this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs); + this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null); } /** @@ -120,9 +135,11 @@ * singleton output object. */ public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix, - boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs outputs) { + boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs outputs, + FreezeTail freezeTail) { this.minSuffixCount1 = minSuffixCount1; this.minSuffixCount2 = minSuffixCount2; + this.freezeTail = freezeTail; this.doShareNonSingletonNodes = doShareNonSingletonNodes; this.shareMaxTailLength = shareMaxTailLength; fst = new FST(inputType, outputs); @@ -179,94 +196,105 @@ return fn; } - private void compilePrevTail(int prefixLenPlus1) throws IOException { - assert prefixLenPlus1 >= 1; - //System.out.println(" compileTail " + prefixLenPlus1); - for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) { - boolean doPrune = false; - boolean doCompile = false; + private void freezeTail(int prefixLenPlus1) throws IOException { + if (freezeTail != null) { + // Custom plugin: + freezeTail.freeze(frontier, prefixLenPlus1, lastInput); + } else { + //System.out.println(" compileTail " + prefixLenPlus1); + for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) { - final UnCompiledNode node = frontier[idx]; - final UnCompiledNode parent = frontier[idx-1]; + boolean doPrune = false; + boolean doCompile = false; - if (node.inputCount < minSuffixCount1) { - doPrune = true; - doCompile = true; - } else if (idx > prefixLenPlus1) { - // prune if parent's inputCount is less than suffixMinCount2 - if (parent.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && parent.inputCount == 1) { - // my parent, about to be compiled, doesn't make the cut, so - // I'm definitely pruned + final UnCompiledNode node = frontier[idx]; + final UnCompiledNode parent = idx == 0 ? null : frontier[idx-1]; - // if pruneCount2 is 1, we keep only up - // until the 'distinguished edge', ie we keep only the - // 'divergent' part of the FST. if my parent, about to be - // compiled, has inputCount 1 then we are already past the - // distinguished edge. NOTE: this only works if - // the FST outputs are not "compressible" (simple - // ords ARE compressible). + // nocommit: just have for loop go to 1 not 0? + if (parent == null) { + return; + } + + if (node.inputCount < minSuffixCount1) { doPrune = true; + doCompile = true; + } else if (idx > prefixLenPlus1) { + //System.out.println(" pic=" + parent.inputCount + " parent=" + parent + " numArcs=" + parent.numArcs); + // prune if parent's inputCount is less than suffixMinCount2 + if (parent.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && parent.inputCount == 1 && idx > 1)) { + // my parent, about to be compiled, doesn't make the cut, so + // I'm definitely pruned + + // if minSuffixCount2 is 1, we keep only up + // until the 'distinguished edge', ie we keep only the + // 'divergent' part of the FST. if my parent, about to be + // compiled, has inputCount 1 then we are already past the + // distinguished edge. NOTE: this only works if + // the FST outputs are not "compressible" (simple + // ords ARE compressible). + doPrune = true; + } else { + // my parent, about to be compiled, does make the cut, so + // I'm definitely not pruned + doPrune = false; + } + doCompile = true; } else { - // my parent, about to be compiled, does make the cut, so - // I'm definitely not pruned - doPrune = false; + // if pruning is disabled (count is 0) we can always + // compile current node + doCompile = minSuffixCount2 == 0; } - doCompile = true; - } else { - // if pruning is disabled (count is 0) we can always - // compile current node - doCompile = minSuffixCount2 == 0; - } - //System.out.println(" label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune); + //System.out.println(" label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune); - if (node.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && node.inputCount == 1) { - // drop all arcs - for(int arcIdx=0;arcIdx target = (UnCompiledNode) node.arcs[arcIdx].target; - target.clear(); + if (node.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && node.inputCount == 1 && idx > 1)) { + // drop all arcs + for(int arcIdx=0;arcIdx target = (UnCompiledNode) node.arcs[arcIdx].target; + target.clear(); + } + node.numArcs = 0; } - node.numArcs = 0; - } - if (doPrune) { - // this node doesn't make it -- deref it - node.clear(); - parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node); - } else { + if (doPrune) { + // this node doesn't make it -- deref it + node.clear(); + parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node); + } else { - if (minSuffixCount2 != 0) { - compileAllTargets(node, lastInput.length-idx); - } - final T nextFinalOutput = node.output; + if (minSuffixCount2 != 0) { + compileAllTargets(node, lastInput.length-idx); + } + final T nextFinalOutput = node.output; - // We "fake" the node as being final if it has no - // outgoing arcs; in theory we could leave it - // as non-final (the FST can represent this), but - // FSTEnum, Util, etc., have trouble w/ non-final - // dead-end states: - final boolean isFinal = node.isFinal || node.numArcs == 0; + // We "fake" the node as being final if it has no + // outgoing arcs; in theory we could leave it + // as non-final (the FST can represent this), but + // FSTEnum, Util, etc., have trouble w/ non-final + // dead-end states: + final boolean isFinal = node.isFinal || node.numArcs == 0; - if (doCompile) { - // this node makes it and we now compile it. first, - // compile any targets that were previously - // undecided: - parent.replaceLast(lastInput.ints[lastInput.offset + idx-1], - compileNode(node, 1+lastInput.length-idx), - nextFinalOutput, - isFinal); - } else { - // replaceLast just to install - // nextFinalOutput/isFinal onto the arc - parent.replaceLast(lastInput.ints[lastInput.offset + idx-1], - node, - nextFinalOutput, - isFinal); - // this node will stay in play for now, since we are - // undecided on whether to prune it. later, it - // will be either compiled or pruned, so we must - // allocate a new node: - frontier[idx] = new UnCompiledNode(this, idx); + if (doCompile) { + // this node makes it and we now compile it. first, + // compile any targets that were previously + // undecided: + parent.replaceLast(lastInput.ints[lastInput.offset + idx-1], + compileNode(node, 1+lastInput.length-idx), + nextFinalOutput, + isFinal); + } else { + // replaceLast just to install + // nextFinalOutput/isFinal onto the arc + parent.replaceLast(lastInput.ints[lastInput.offset + idx-1], + node, + nextFinalOutput, + isFinal); + // this node will stay in play for now, since we are + // undecided on whether to prune it. later, it + // will be either compiled or pruned, so we must + // allocate a new node: + frontier[idx] = new UnCompiledNode(this, idx); + } } } } @@ -324,7 +352,23 @@ * different outputs, as long as outputs impls the merge * method. */ public void add(IntsRef input, T output) throws IOException { - //System.out.println("\nFST ADD: input=" + input + " output=" + fst.outputs.outputToString(output)); + // nocommit + { + BytesRef b = new BytesRef(input.length); + for(int x=0;x { + public static class Arc { public int label; // really an "unsigned" byte public Node target; public boolean isFinal; @@ -502,16 +539,20 @@ } } - static final class UnCompiledNode implements Node { + public static final class UnCompiledNode implements Node { final Builder owner; - int numArcs; - Arc[] arcs; - T output; - boolean isFinal; - long inputCount; + public int numArcs; + public Arc[] arcs; + // TODO: instead of recording isFinal/output on the + // node, maybe we should use -1 arc to mean "end" (like + // we do when reading the FST). Would simplify much + // code here... + public T output; + public boolean isFinal; + public long inputCount; /** This node's depth, starting from the automaton root. */ - final int depth; + public final int depth; /** * @param depth Index: lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java --- lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java Mon Aug 01 12:09:21 2011 -0400 @@ -133,6 +133,6 @@ @Override public String outputToString(BytesRef output) { - return output.utf8ToString(); + return output.toString(); } } Index: lucene/src/java/org/apache/lucene/util/fst/FST.java --- lucene/src/java/org/apache/lucene/util/fst/FST.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/util/fst/FST.java Mon Aug 01 12:09:21 2011 -0400 @@ -123,7 +123,7 @@ public int label; public T output; - int target; + public int target; byte flags; public T nextFinalOutput; @@ -274,6 +274,10 @@ } } + public T getEmptyOutput() { + return emptyOutput; + } + void setEmptyOutput(T v) throws IOException { if (emptyOutput != null) { emptyOutput = outputs.merge(emptyOutput, v); @@ -597,9 +601,9 @@ arc.label = END_LABEL; arc.output = follow.nextFinalOutput; if (follow.target <= 0) { - arc.flags = BIT_LAST_ARC; + arc.flags = BIT_LAST_ARC | BIT_FINAL_ARC; } else { - arc.flags = 0; + arc.flags = BIT_FINAL_ARC; arc.nextArc = follow.target; } //System.out.println(" insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output)); @@ -609,8 +613,7 @@ } } - // Not private because NodeHash needs access: - Arc readFirstRealArc(int address, Arc arc) throws IOException { + public Arc readFirstRealArc(int address, Arc arc) throws IOException { final BytesReader in = getBytesReader(address); @@ -693,7 +696,9 @@ return readLabel(in); } - Arc readNextRealArc(Arc arc, final BytesReader in) throws IOException { + /** Never returns null, but you should never call this if + * arc.isLast() is true. */ + public Arc readNextRealArc(Arc arc, final BytesReader in) throws IOException { // this is a continuing arc in a fixed array if (arc.bytesPerArc != 0) { // arcs are at fixed entries @@ -925,7 +930,7 @@ } } - final BytesReader getBytesReader(int pos) { + public final BytesReader getBytesReader(int pos) { // TODO: maybe re-use via ThreadLocal? return new BytesReader(pos); } Index: lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java --- lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/util/fst/UpToTwoPositiveIntOutputs.java Mon Aug 01 12:09:21 2011 -0400 @@ -39,8 +39,8 @@ public final class UpToTwoPositiveIntOutputs extends Outputs { public final static class TwoLongs { - final long first; - final long second; + public final long first; + public final long second; public TwoLongs(long first, long second) { this.first = first; Index: lucene/src/java/org/apache/lucene/util/fst/Util.java --- lucene/src/java/org/apache/lucene/util/fst/Util.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/java/org/apache/lucene/util/fst/Util.java Mon Aug 01 12:09:21 2011 -0400 @@ -213,6 +213,7 @@ // Shape for states. final String stateShape = "circle"; + final String finalStateShape = "doublecircle"; // Emit DOT prologue. out.write("digraph FST {\n"); @@ -223,12 +224,34 @@ } emitDotState(out, "initial", "point", "white", ""); - emitDotState(out, Integer.toString(startArc.target), stateShape, - fst.isExpandedTarget(startArc) ? expandedNodeColor : null, - ""); + + final T NO_OUTPUT = fst.outputs.getNoOutput(); + + final FST.Arc scratchArc = new FST.Arc(); + + { + final String stateColor; + if (fst.isExpandedTarget(startArc)) { + stateColor = expandedNodeColor; + } else { + stateColor = null; + } + + final boolean isFinal; + final T finalOutput; + if (startArc.isFinal()) { + isFinal = true; + finalOutput = startArc.nextFinalOutput == NO_OUTPUT ? null : startArc.nextFinalOutput; + } else { + isFinal = false; + finalOutput = null; + } + + emitDotState(out, Integer.toString(startArc.target), isFinal ? finalStateShape : stateShape, stateColor, finalOutput == null ? "" : fst.outputs.outputToString(finalOutput)); + } + out.write(" initial -> " + startArc.target + "\n"); - final T NO_OUTPUT = fst.outputs.getNoOutput(); int level = 0; while (!nextLevelQueue.isEmpty()) { @@ -240,19 +263,48 @@ out.write("\n // Transitions and states at level: " + level + "\n"); while (!thisLevelQueue.isEmpty()) { final FST.Arc arc = thisLevelQueue.remove(thisLevelQueue.size() - 1); - if (fst.targetHasArcs(arc)) { // scan all arcs final int node = arc.target; fst.readFirstTargetArc(arc, arc); - + + if (arc.label == FST.END_LABEL) { + // Skip it -- prior recursion took this into account already + assert !arc.isLast(); + fst.readNextArc(arc); + } + while (true) { + // Emit the unseen state and add it to the queue for the next level. if (arc.target >= 0 && !seen.get(arc.target)) { - final boolean isExpanded = fst.isExpandedTarget(arc); - emitDotState(out, Integer.toString(arc.target), stateShape, - isExpanded ? expandedNodeColor : null, - labelStates ? Integer.toString(arc.target) : ""); + + /* + boolean isFinal = false; + T finalOutput = null; + fst.readFirstTargetArc(arc, scratchArc); + if (scratchArc.isFinal() && fst.targetHasArcs(scratchArc)) { + // target is final + isFinal = true; + finalOutput = scratchArc.output == NO_OUTPUT ? null : scratchArc.output; + System.out.println("dot hit final label=" + (char) scratchArc.label); + } + */ + final String stateColor; + if (fst.isExpandedTarget(arc)) { + stateColor = expandedNodeColor; + } else { + stateColor = null; + } + + final String finalOutput; + if (arc.nextFinalOutput != null && arc.nextFinalOutput != NO_OUTPUT) { + finalOutput = fst.outputs.outputToString(arc.nextFinalOutput); + } else { + finalOutput = ""; + } + + emitDotState(out, Integer.toString(arc.target), arc.isFinal() ? finalStateShape : stateShape, stateColor, finalOutput); seen.set(arc.target); nextLevelQueue.add(new FST.Arc().copyFrom(arc)); sameLevelStates.add(arc.target); @@ -265,15 +317,20 @@ outs = ""; } - final String cl; - if (arc.label == FST.END_LABEL) { - cl = "~"; - } else { - cl = printableLabel(arc.label); + if (!fst.targetHasArcs(arc) && arc.isFinal() && arc.nextFinalOutput != NO_OUTPUT) { + // nocommit -- this is broken again? + // Tricky special case: sometimes, due to + // pruning, the builder can [sillily] produce + // an FST with an arc into the final end state + // (-1) but also with a next final output; in + // this case we pull that output up onto this + // arc + outs = outs + "/[" + fst.outputs.outputToString(arc.nextFinalOutput) + "]"; } - out.write(" " + node + " -> " + arc.target + " [label=\"" + cl + outs + "\"]\n"); - + assert arc.label != FST.END_LABEL; + out.write(" " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"]\n"); + // Break the loop if we're on the last arc of this state. if (arc.isLast()) { break; @@ -295,7 +352,7 @@ } // Emit terminating state (always there anyway). - out.write(" -1 [style=filled, color=black, shape=circle, label=\"\"]\n\n"); + out.write(" -1 [style=filled, color=black, shape=doublecircle, label=\"\"]\n\n"); out.write(" {rank=sink; -1 }\n"); out.write("}\n"); Index: lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java --- lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Mon Aug 01 12:09:21 2011 -0400 @@ -60,7 +60,7 @@ private final Random r; - public MockIndexWriter(Random r,Directory dir, IndexWriterConfig conf) throws IOException { + public MockIndexWriter(Random r, Directory dir, IndexWriterConfig conf) throws IOException { super(dir, conf); // must make a private random since our methods are // called from different threads; else test failures may Index: lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java --- lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java Mon Aug 01 12:09:21 2011 -0400 @@ -30,6 +30,8 @@ import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.codecs.BlockTreeTermsReader; +import org.apache.lucene.index.codecs.BlockTreeTermsWriter; import org.apache.lucene.index.codecs.BlockTermsReader; import org.apache.lucene.index.codecs.BlockTermsWriter; import org.apache.lucene.index.codecs.Codec; @@ -43,6 +45,8 @@ import org.apache.lucene.index.codecs.PerDocValues; import org.apache.lucene.index.codecs.PostingsReaderBase; import org.apache.lucene.index.codecs.PostingsWriterBase; +import org.apache.lucene.index.codecs.BlockTreePostingsReaderBase; +import org.apache.lucene.index.codecs.BlockTreePostingsWriterBase; import org.apache.lucene.index.codecs.TermStats; import org.apache.lucene.index.codecs.TermsIndexReaderBase; import org.apache.lucene.index.codecs.TermsIndexWriterBase; @@ -53,6 +57,8 @@ import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory; import org.apache.lucene.index.codecs.pulsing.PulsingPostingsReaderImpl; import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl; +import org.apache.lucene.index.codecs.pulsingtree.PulsingTreePostingsReader; +import org.apache.lucene.index.codecs.pulsingtree.PulsingTreePostingsWriter; import org.apache.lucene.index.codecs.sep.IntIndexInput; import org.apache.lucene.index.codecs.sep.IntIndexOutput; import org.apache.lucene.index.codecs.sep.IntStreamFactory; @@ -60,6 +66,8 @@ import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl; import org.apache.lucene.index.codecs.standard.StandardPostingsReader; import org.apache.lucene.index.codecs.standard.StandardPostingsWriter; +import org.apache.lucene.index.codecs.standardtree.StandardTreePostingsReader; +import org.apache.lucene.index.codecs.standardtree.StandardTreePostingsWriter; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; @@ -137,7 +145,7 @@ final long seed = seedRandom.nextLong(); if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: writing to seg=" + state.segmentName + " seed=" + seed); + System.out.println("MockRandomCodec: writing to seg=" + state.segmentName + " codecID=" + state.codecId + " seed=" + seed); } final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SEED_EXT); @@ -152,87 +160,122 @@ random.nextInt(); // consume a random for buffersize - PostingsWriterBase postingsWriter; + if (random.nextBoolean()) { + // Use BlockTree terms dict - if (random.nextBoolean()) { - postingsWriter = new SepPostingsWriterImpl(state, new MockIntStreamFactory(random), skipInterval); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: writing BlockTree terms dict"); + } + BlockTreePostingsWriterBase postingsWriter = new StandardTreePostingsWriter(state); + + if (random.nextBoolean()) { + final int totTFCutoff = _TestUtil.nextInt(random, 1, 20); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: pulsing postings with totTFCutoff=" + totTFCutoff); + } + postingsWriter = new PulsingTreePostingsWriter(totTFCutoff, postingsWriter); + } + + final int minTermsInBlock = _TestUtil.nextInt(random, 4, 100); + final int maxTermsInBlock = minTermsInBlock*2 + random.nextInt(100); + + boolean success = false; + try { + FieldsConsumer ret = new BlockTreeTermsWriter(state, postingsWriter, minTermsInBlock, maxTermsInBlock); + success = true; + return ret; + } finally { + if (!success) { + postingsWriter.close(); + } + } } else { + if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: writing Standard postings"); + System.out.println("MockRandomCodec: writing Block terms dict"); } - postingsWriter = new StandardPostingsWriter(state, skipInterval); - } - if (random.nextBoolean()) { - final int totTFCutoff = _TestUtil.nextInt(random, 1, 20); - if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: pulsing postings with totTFCutoff=" + totTFCutoff); + PostingsWriterBase postingsWriter; + if (random.nextBoolean()) { + postingsWriter = new SepPostingsWriterImpl(state, new MockIntStreamFactory(random), skipInterval); + } else { + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: writing Standard postings"); + } + postingsWriter = new StandardPostingsWriter(state, skipInterval); } - postingsWriter = new PulsingPostingsWriterImpl(totTFCutoff, postingsWriter); - } - final TermsIndexWriterBase indexWriter; - boolean success = false; + if (random.nextBoolean()) { + final int totTFCutoff = _TestUtil.nextInt(random, 1, 20); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: writing pulsing postings with totTFCutoff=" + totTFCutoff); + } + postingsWriter = new PulsingPostingsWriterImpl(totTFCutoff, postingsWriter); + } - try { - if (random.nextBoolean()) { - state.termIndexInterval = _TestUtil.nextInt(random, 1, 100); - if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: fixed-gap terms index (tii=" + state.termIndexInterval + ")"); + boolean success = false; + + final TermsIndexWriterBase indexWriter; + try { + if (random.nextBoolean()) { + state.termIndexInterval = _TestUtil.nextInt(random, 1, 100); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: fixed-gap terms index (tii=" + state.termIndexInterval + ")"); + } + indexWriter = new FixedGapTermsIndexWriter(state); + } else { + final VariableGapTermsIndexWriter.IndexTermSelector selector; + final int n2 = random.nextInt(3); + if (n2 == 0) { + final int tii = _TestUtil.nextInt(random, 1, 100); + selector = new VariableGapTermsIndexWriter.EveryNTermSelector(tii); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: variable-gap terms index (tii=" + tii + ")"); + } + } else if (n2 == 1) { + final int docFreqThresh = _TestUtil.nextInt(random, 2, 100); + final int tii = _TestUtil.nextInt(random, 1, 100); + selector = new VariableGapTermsIndexWriter.EveryNOrDocFreqTermSelector(docFreqThresh, tii); + } else { + final long seed2 = random.nextLong(); + final int gap = _TestUtil.nextInt(random, 2, 40); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: random-gap terms index (max gap=" + gap + ")"); + } + selector = new VariableGapTermsIndexWriter.IndexTermSelector() { + final Random rand = new Random(seed2); + + @Override + public boolean isIndexTerm(BytesRef term, TermStats stats) { + return rand.nextInt(gap) == gap/2; + } + + @Override + public void newField(FieldInfo fieldInfo) { + } + }; + } + indexWriter = new VariableGapTermsIndexWriter(state, selector); + success = true; } - indexWriter = new FixedGapTermsIndexWriter(state); - } else { - final VariableGapTermsIndexWriter.IndexTermSelector selector; - final int n2 = random.nextInt(3); - if (n2 == 0) { - final int tii = _TestUtil.nextInt(random, 1, 100); - selector = new VariableGapTermsIndexWriter.EveryNTermSelector(tii); - if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: variable-gap terms index (tii=" + tii + ")"); + } finally { + if (!success) { + postingsWriter.close(); + } + } + + success = false; + try { + FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter); + success = true; + return ret; + } finally { + if (!success) { + try { + postingsWriter.close(); + } finally { + indexWriter.close(); } - } else if (n2 == 1) { - final int docFreqThresh = _TestUtil.nextInt(random, 2, 100); - final int tii = _TestUtil.nextInt(random, 1, 100); - selector = new VariableGapTermsIndexWriter.EveryNOrDocFreqTermSelector(docFreqThresh, tii); - } else { - final long seed2 = random.nextLong(); - final int gap = _TestUtil.nextInt(random, 2, 40); - if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: random-gap terms index (max gap=" + gap + ")"); - } - selector = new VariableGapTermsIndexWriter.IndexTermSelector() { - final Random rand = new Random(seed2); - - @Override - public boolean isIndexTerm(BytesRef term, TermStats stats) { - return rand.nextInt(gap) == gap/2; - } - - @Override - public void newField(FieldInfo fieldInfo) { - } - }; - } - indexWriter = new VariableGapTermsIndexWriter(state, selector); - } - success = true; - } finally { - if (!success) { - postingsWriter.close(); - } - } - - success = false; - try { - FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter); - success = true; - return ret; - } finally { - if (!success) { - try { - postingsWriter.close(); - } finally { - indexWriter.close(); } } } @@ -245,7 +288,7 @@ final IndexInput in = state.dir.openInput(seedFileName, state.context); final long seed = in.readLong(); if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " seed=" + seed); + System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " codecID=" + state.codecId + " seed=" + seed); } in.close(); @@ -256,91 +299,135 @@ System.out.println("MockRandomCodec: readBufferSize=" + readBufferSize); } - PostingsReaderBase postingsReader; + if (random.nextBoolean()) { + // Use BlockTree terms dict + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: reading BlockTree terms dict"); + } + BlockTreePostingsReaderBase postingsReader = new StandardTreePostingsReader(state.dir, state.segmentInfo, state.context, state.codecId); - if (random.nextBoolean()) { - postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo, - state.context, new MockIntStreamFactory(random), state.codecId); + if (random.nextBoolean()) { + final int totTFCutoff = _TestUtil.nextInt(random, 1, 20); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: reading pulsing postings with totTFCutoff=" + totTFCutoff); + } + postingsReader = new PulsingTreePostingsReader(postingsReader); + } + + // randomness diverges from writer, here: + if (state.termsIndexDivisor != -1) { + state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10); + } + + final int termsCacheSize = _TestUtil.nextInt(random, 1, 1024); + + boolean success = false; + try { + FieldsProducer ret = new BlockTreeTermsReader(state.dir, + state.fieldInfos, + state.segmentInfo.name, + postingsReader, + state.context, + termsCacheSize, + state.codecId, + state.termsIndexDivisor); + success = true; + return ret; + } finally { + if (!success) { + postingsReader.close(); + } + } } else { - if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: reading Standard postings"); + + PostingsReaderBase postingsReader; + + if (random.nextBoolean()) { + postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo, + state.context, new MockIntStreamFactory(random), state.codecId); + } else { + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: reading Standard postings"); + } + postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId); } - postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId); - } - if (random.nextBoolean()) { - final int totTFCutoff = _TestUtil.nextInt(random, 1, 20); - if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: reading pulsing postings with totTFCutoff=" + totTFCutoff); + if (random.nextBoolean()) { + final int totTFCutoff = _TestUtil.nextInt(random, 1, 20); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: reading pulsing postings with totTFCutoff=" + totTFCutoff); + } + postingsReader = new PulsingPostingsReaderImpl(postingsReader); } - postingsReader = new PulsingPostingsReaderImpl(postingsReader); - } - final TermsIndexReaderBase indexReader; - boolean success = false; + final TermsIndexReaderBase indexReader; + boolean success = false; + try { + final boolean doFixedGap = random.nextBoolean(); - try { - if (random.nextBoolean()) { - // if termsIndexDivisor is set to -1, we should not touch it. It means a - // test explicitly instructed not to load the terms index. + // randomness diverges from writer, here: if (state.termsIndexDivisor != -1) { state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10); } - if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: fixed-gap terms index (divisor=" + state.termsIndexDivisor + ")"); + + if (doFixedGap) { + // if termsIndexDivisor is set to -1, we should not touch it. It means a + // test explicitly instructed not to load the terms index. + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: fixed-gap terms index (divisor=" + state.termsIndexDivisor + ")"); + } + indexReader = new FixedGapTermsIndexReader(state.dir, + state.fieldInfos, + state.segmentInfo.name, + state.termsIndexDivisor, + BytesRef.getUTF8SortedAsUnicodeComparator(), + state.codecId, state.context); + } else { + final int n2 = random.nextInt(3); + if (n2 == 1) { + random.nextInt(); + } else if (n2 == 2) { + random.nextLong(); + } + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: variable-gap terms index (divisor=" + state.termsIndexDivisor + ")"); + } + indexReader = new VariableGapTermsIndexReader(state.dir, + state.fieldInfos, + state.segmentInfo.name, + state.termsIndexDivisor, + state.codecId, state.context); + } - indexReader = new FixedGapTermsIndexReader(state.dir, - state.fieldInfos, - state.segmentInfo.name, - state.termsIndexDivisor, - BytesRef.getUTF8SortedAsUnicodeComparator(), - state.codecId, state.context); - } else { - final int n2 = random.nextInt(3); - if (n2 == 1) { - random.nextInt(); - } else if (n2 == 2) { - random.nextLong(); + + success = true; + } finally { + if (!success) { + postingsReader.close(); } - if (LuceneTestCase.VERBOSE) { - System.out.println("MockRandomCodec: variable-gap terms index (divisor=" + state.termsIndexDivisor + ")"); - } - if (state.termsIndexDivisor != -1) { - state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10); - } - indexReader = new VariableGapTermsIndexReader(state.dir, - state.fieldInfos, - state.segmentInfo.name, - state.termsIndexDivisor, - state.codecId, state.context); } - success = true; - } finally { - if (!success) { - postingsReader.close(); - } - } - final int termsCacheSize = _TestUtil.nextInt(random, 1, 1024); + final int termsCacheSize = _TestUtil.nextInt(random, 1, 1024); - success = false; - try { - FieldsProducer ret = new BlockTermsReader(indexReader, - state.dir, - state.fieldInfos, - state.segmentInfo.name, - postingsReader, - state.context, - termsCacheSize, - state.codecId); - success = true; - return ret; - } finally { - if (!success) { - try { - postingsReader.close(); - } finally { - indexReader.close(); + success = false; + try { + FieldsProducer ret = new BlockTermsReader(indexReader, + state.dir, + state.fieldInfos, + state.segmentInfo.name, + postingsReader, + state.context, + termsCacheSize, + state.codecId); + success = true; + return ret; + } finally { + if (!success) { + try { + postingsReader.close(); + } finally { + indexReader.close(); + } } } } @@ -353,6 +440,7 @@ SepPostingsReaderImpl.files(segmentInfo, codecId, files); StandardPostingsReader.files(dir, segmentInfo, codecId, files); BlockTermsReader.files(dir, segmentInfo, codecId, files); + BlockTreeTermsReader.files(dir, segmentInfo, codecId, files); FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files); VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files); DefaultDocValuesConsumer.files(dir, segmentInfo, codecId, files, getDocValuesUseCFS()); @@ -371,6 +459,7 @@ public void getExtensions(Set extensions) { SepPostingsWriterImpl.getExtensions(extensions); BlockTermsReader.getExtensions(extensions); + BlockTreeTermsReader.getExtensions(extensions); FixedGapTermsIndexReader.getIndexExtensions(extensions); VariableGapTermsIndexReader.getIndexExtensions(extensions); DefaultDocValuesConsumer.getDocValuesExtensions(extensions, getDocValuesUseCFS()); Index: lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java --- lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Mon Aug 01 12:09:21 2011 -0400 @@ -348,7 +348,8 @@ private static List testClassesRun = new ArrayList(); private static void initRandom() { - assert !random.initialized; + // nocommit + //assert !random.initialized; staticSeed = "random".equals(TEST_SEED) ? seedRand.nextLong() : TwoLongs.fromString(TEST_SEED).l1; random.setSeed(staticSeed); random.initialized = true; @@ -484,7 +485,7 @@ System.err.println("NOTE: test params are: codec=" + codecDescription + ", locale=" + locale + ", timezone=" + (timeZone == null ? "(null)" : timeZone.getID())); - if (testsFailed) { + if (VERBOSE || testsFailed) { System.err.println("NOTE: all tests run in this JVM:"); System.err.println(Arrays.toString(testClassesRun.toArray())); System.err.println("NOTE: " + System.getProperty("os.name") + " " @@ -1429,10 +1430,11 @@ @Override protected List computeTestMethods() { + // nocommit + initRandom(); if (testMethods != null) return testMethods; - initRandom(); Random r = new Random(random.nextLong()); testClassesRun.add(getTestClass().getJavaClass().getSimpleName()); Index: lucene/src/test/org/apache/lucene/index/TestCodecs.java --- lucene/src/test/org/apache/lucene/index/TestCodecs.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/test/org/apache/lucene/index/TestCodecs.java Mon Aug 01 12:09:21 2011 -0400 @@ -504,6 +504,9 @@ } // Test seek to non-existent terms: + if (VERBOSE) { + System.out.println("TEST: seek non-exist terms"); + } for(int i=0;i<100;i++) { final String text2 = _TestUtil.randomUnicodeString(random) + "."; status = termsEnum.seekCeil(new BytesRef(text2)); @@ -512,6 +515,9 @@ } // Seek to each term, backwards: + if (VERBOSE) { + System.out.println("TEST: seek terms backwards"); + } for(int i=field.terms.length-1;i>=0;i--) { assertEquals(Thread.currentThread().getName() + ": field=" + field.fieldInfo.name + " term=" + field.terms[i].text2, TermsEnum.SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(field.terms[i].text2))); assertEquals(field.terms[i].docs.length, termsEnum.docFreq()); Index: lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java --- lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java Mon Aug 01 12:09:21 2011 -0400 @@ -462,7 +462,7 @@ */ if (VERBOSE) { - System.out.println("TEST: verify prefix=" + prefixRef.utf8ToString()); + System.out.println("TEST: verify prefix=" + (prefixRef==null ? "null" : prefixRef.utf8ToString())); System.out.println("TEST: all TERMS:"); TermsEnum allTE = MultiFields.getTerms(r, "field").iterator(); int ord = 0; Index: lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java --- lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Mon Aug 01 12:09:21 2011 -0400 @@ -169,7 +169,7 @@ Set dif = difFiles(files, files2); if (!Arrays.equals(files, files2)) { - fail("IndexFileDeleter failed to delete unreferenced extra files: should have deleted " + (filesPre.length-files.length) + " files but only deleted " + (filesPre.length - files2.length) + "; expected files:\n " + asString(files) + "\n actual files:\n " + asString(files2)+"\ndif: "+dif); + fail("IndexFileDeleter failed to delete unreferenced extra files: should have deleted " + (filesPre.length-files.length) + " files but only deleted " + (filesPre.length - files2.length) + "; expected files:\n " + asString(files) + "\n actual files:\n " + asString(files2)+"\ndiff: "+dif); } } Index: lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java --- lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Mon Aug 01 12:09:21 2011 -0400 @@ -74,8 +74,15 @@ Term term = new Term("city", "Amsterdam"); int hitCount = getHitCount(dir, term); assertEquals(1, hitCount); + if (VERBOSE) { + System.out.println("\nTEST: now delete by term=" + term); + } modifier.deleteDocuments(term); modifier.commit(); + + if (VERBOSE) { + System.out.println("\nTEST: now getHitCount"); + } hitCount = getHitCount(dir, term); assertEquals(0, hitCount); Index: lucene/src/test/org/apache/lucene/index/TestMultiFields.java --- lucene/src/test/org/apache/lucene/index/TestMultiFields.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/test/org/apache/lucene/index/TestMultiFields.java Mon Aug 01 12:09:21 2011 -0400 @@ -29,10 +29,15 @@ int num = atLeast(2); for (int iter = 0; iter < num; iter++) { + if (VERBOSE) { + System.out.println("TEST: iter=" + iter); + } + Directory dir = newDirectory(); IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(NoMergePolicy.COMPOUND_FILES)); _TestUtil.keepFullyDeletedSegments(w); + w.setInfoStream(VERBOSE ? System.out : null); Map> docs = new HashMap>(); Set deleted = new HashSet(); @@ -46,6 +51,9 @@ doc.add(id); boolean onlyUniqueTerms = random.nextBoolean(); + if (VERBOSE) { + System.out.println("TEST: onlyUniqueTerms=" + onlyUniqueTerms + " numDocs=" + numDocs); + } Set uniqueTerms = new HashSet(); for(int i=0;i termsList = new ArrayList(uniqueTerms); Collections.sort(termsList, BytesRef.getUTF8SortedAsUTF16Comparator()); - System.out.println("UTF16 order:"); + System.out.println("TEST: terms in UTF16 order:"); for(BytesRef b : termsList) { - System.out.println(" " + UnicodeUtil.toHexString(b.utf8ToString())); + System.out.println(" " + UnicodeUtil.toHexString(b.utf8ToString()) + " " + b); + for(int docID : docs.get(b)) { + if (deleted.contains(docID)) { + System.out.println(" " + docID + " (deleted)"); + } else { + System.out.println(" " + docID); + } + } } } IndexReader reader = w.getReader(); w.close(); - //System.out.println("TEST reader=" + reader); + if (VERBOSE) { + System.out.println("TEST: reader=" + reader); + } Bits liveDocs = MultiFields.getLiveDocs(reader); for(int delDoc : deleted) { @@ -99,7 +119,7 @@ for(int i=0;i<100;i++) { BytesRef term = terms.get(random.nextInt(terms.size())); if (VERBOSE) { - System.out.println("TEST: seek to term= "+ UnicodeUtil.toHexString(term.utf8ToString())); + System.out.println("TEST: seek term="+ UnicodeUtil.toHexString(term.utf8ToString()) + " " + term); } DocsEnum docsEnum = terms2.docs(liveDocs, term, null); Index: lucene/src/test/org/apache/lucene/index/TestNRTThreads.java --- lucene/src/test/org/apache/lucene/index/TestNRTThreads.java Mon Aug 01 12:07:14 2011 -0400 +++ lucene/src/test/org/apache/lucene/index/TestNRTThreads.java Mon Aug 01 12:09:21 2011 -0400 @@ -132,8 +132,9 @@ } _TestUtil.reduceOpenFiles(writer); - final int NUM_INDEX_THREADS = 2; - final int NUM_SEARCH_THREADS = 3; + // nocommit + final int NUM_INDEX_THREADS = 1; // 2 + final int NUM_SEARCH_THREADS = 1; // 3 final int RUN_TIME_SEC = LuceneTestCase.TEST_NIGHTLY ? 300 : RANDOM_MULTIPLIER; Index: lucene/src/test/org/apache/lucene/index/codecs/TestBlockTree.java --- /dev/null Thu Jan 01 00:00:00 1970 +0000 +++ lucene/src/test/org/apache/lucene/index/codecs/TestBlockTree.java Mon Aug 01 12:09:21 2011 -0400 @@ -0,0 +1,383 @@ +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.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.SegmentReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.index.codecs.standardtree.StandardTreeCodec; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.RAMDirectory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util._TestUtil; + +// nocommit rename to TestSomethingTermsDict +// nocommit fix test to also test postings! + +public class TestBlockTree extends LuceneTestCase { + private Directory d; + private IndexReader r; + + private final String FIELD = "field"; + + private IndexReader makeIndex(int minTermsInBlock, int maxTermsInBlock, String... terms) throws Exception { + // nocommit -- cutover to newDirectory + d = new RAMDirectory(); + // nocommit -- switch to riw / other codecs: + IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)); + + CoreCodecProvider cp = new CoreCodecProvider(); + cp.unregister(cp.lookup("StandardTree")); + cp.register(new StandardTreeCodec(minTermsInBlock, maxTermsInBlock)); + cp.setDefaultFieldCodec("StandardTree"); + iwc.setCodecProvider(cp); + + final IndexWriter w = new IndexWriter(d, iwc); + w.setInfoStream(VERBOSE ? System.out : null); + for(String term : terms) { + Document doc = new Document(); + // nocommit -- switch to newField + Field f = new Field(FIELD, term, Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS); + doc.add(f); + w.addDocument(doc); + } + if (r != null) { + close(); + } + r = IndexReader.open(w, true); + w.close(); + return r; + } + + private void close() throws Exception { + final Directory d = ((SegmentReader) r.getSequentialSubReaders()[0]).directory(); + r.close(); + d.close(); + } + + private int docFreq(IndexReader r, String term) throws Exception { + return r.docFreq(new Term(FIELD, term)); + } + + public void testEasy() throws Exception { + // No floor arcs: + r = makeIndex(3, 6, "aa0", "aa1", "aa2", "aa3", "bb0", "bb1", "bb2", "bb3", "aa"); + + // First term in block: + assertEquals(1, docFreq(r, "aa0")); + + // Scan forward to another term in same block + assertEquals(1, docFreq(r, "aa2")); + + assertEquals(1, docFreq(r, "aa")); + + // Reset same block then scan forwards + assertEquals(1, docFreq(r, "aa1")); + + // Not found, in same block + assertEquals(0, docFreq(r, "aa5")); + + // Found, in same block + assertEquals(1, docFreq(r, "aa2")); + + // Not found in index: + assertEquals(0, docFreq(r, "b0")); + + // Found: + assertEquals(1, docFreq(r, "aa2")); + + // Found, rewind: + assertEquals(1, docFreq(r, "aa0")); + + + // First term in block: + assertEquals(1, docFreq(r, "bb0")); + + // Scan forward to another term in same block + assertEquals(1, docFreq(r, "bb2")); + + // Reset same block then scan forwards + assertEquals(1, docFreq(r, "bb1")); + + // Not found, in same block + assertEquals(0, docFreq(r, "bb5")); + + // Found, in same block + assertEquals(1, docFreq(r, "bb2")); + + // Not found in index: + assertEquals(0, docFreq(r, "b0")); + + // Found: + assertEquals(1, docFreq(r, "bb2")); + + // Found, rewind: + assertEquals(1, docFreq(r, "bb0")); + + close(); + } + + // tests: + // - test same prefix has non-floor block and floor block (ie, has 2 long outputs on same term prefix) + // - term that's entirely in the index + + public void testFloorBlocks() throws Exception { + // nocommit put 'aa' back! + final String[] terms = new String[] {"aa0", "aa1", "aa2", "aa3", "aa4", "aa5", "aa6", "aa7", "aa8", "aa9", "aa", "xx"}; + r = makeIndex(3, 6, terms); + //r = makeIndex(3, 6, "aa0", "aa1", "aa2", "aa3", "aa4", "aa5", "aa6", "aa7", "aa8", "aa9"); + + // First term in first block: + assertEquals(1, docFreq(r, "aa0")); + assertEquals(1, docFreq(r, "aa4")); + + // No block + assertEquals(0, docFreq(r, "bb0")); + + // Second block + assertEquals(1, docFreq(r, "aa4")); + + // Backwards to prior floor block: + assertEquals(1, docFreq(r, "aa0")); + + // Forwards to last floor block: + assertEquals(1, docFreq(r, "aa9")); + + assertEquals(0, docFreq(r, "a")); + assertEquals(1, docFreq(r, "aa")); + assertEquals(0, docFreq(r, "a")); + assertEquals(1, docFreq(r, "aa")); + + // Forwards to last floor block: + assertEquals(1, docFreq(r, "xx")); + assertEquals(1, docFreq(r, "aa1")); + assertEquals(0, docFreq(r, "yy")); + + assertEquals(1, docFreq(r, "xx")); + assertEquals(1, docFreq(r, "aa9")); + + assertEquals(1, docFreq(r, "xx")); + assertEquals(1, docFreq(r, "aa4")); + + final TermsEnum te = r.getSequentialSubReaders()[0].fields().terms(FIELD).iterator(); + while(te.next() != null) { + //System.out.println("TEST: next term=" + te.term().utf8ToString()); + } + + assertTrue(seekExact(te, "aa1")); + assertEquals("aa2", next(te)); + assertTrue(seekExact(te, "aa8")); + assertEquals("aa9", next(te)); + assertEquals("xx", next(te)); + + testRandomSeeks(r, terms); + close(); + } + + // nocommit: test 0 terms case too! + + private String getRandomString() { + // nocommit + //return _TestUtil.randomSimpleString(random); + return _TestUtil.randomRealisticUnicodeString(random); + } + + public void testRandomTerms() throws Exception { + // nocommit + //final String[] terms = new String[_TestUtil.nextInt(random, 1, atLeast(1000))]; + final String[] terms = new String[_TestUtil.nextInt(random, 1, atLeast(500))]; + final Set seen = new HashSet(); + + final boolean allowEmptyString = random.nextBoolean(); + + if (random.nextInt(10) == 7 && terms.length > 2) { + // Sometimes add a bunch of terms sharing a longish common prefix: + final int numTermsSamePrefix = random.nextInt(terms.length/2); + if (numTermsSamePrefix > 0) { + String prefix; + while(true) { + prefix = getRandomString(); + if (prefix.length() < 5) { + continue; + } else { + break; + } + } + while(seen.size() < numTermsSamePrefix) { + final String t = prefix + getRandomString(); + if (!seen.contains(t)) { + terms[seen.size()] = t; + seen.add(t); + } + } + } + } + + while(seen.size() < terms.length) { + // nocommit -- use full unicode string + final String t = getRandomString(); + if (!seen.contains(t) && (allowEmptyString || t.length() != 0)) { + terms[seen.size()] = t; + seen.add(t); + } + } + final int minBlockSize = _TestUtil.nextInt(random, 1, 10); + final int maxBlockSize = Math.max(2*(minBlockSize-1) + random.nextInt(60), 1); + if (VERBOSE) { + System.out.println("TEST: minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize); + } + r = makeIndex(minBlockSize, maxBlockSize, terms); + testRandomSeeks(r, terms); + close(); + } + + // sugar + private boolean seekExact(TermsEnum te, String term) throws IOException { + return te.seekExact(new BytesRef(term), random.nextBoolean()); + } + + // sugar + private String next(TermsEnum te) throws IOException { + final BytesRef br = te.next(); + if (br == null) { + return null; + } else { + return br.utf8ToString(); + } + } + + private BytesRef getNonExistTerm(BytesRef[] terms) { + BytesRef t = null; + while(true) { + final String ts = getRandomString(); + t = new BytesRef(ts); + if (Arrays.binarySearch(terms, t) < 0) { + return t; + } + } + } + + private void testRandomSeeks(IndexReader r, String... validTermStrings) throws IOException { + final BytesRef[] validTerms = new BytesRef[validTermStrings.length]; + for(int termIDX=0;termIDX= 0, te.seekExact(t, random.nextBoolean())); + } else { + if (VERBOSE) { + System.out.println(" seekCeil"); + } + + final TermsEnum.SeekStatus result = te.seekCeil(t, random.nextBoolean()); + if (VERBOSE) { + System.out.println(" got " + result); + } + + if (loc >= 0) { + assertEquals(TermsEnum.SeekStatus.FOUND, result); + } else if (loc == END_LOC) { + assertEquals(TermsEnum.SeekStatus.END, result); + } else { + assert loc >= -validTerms.length; + assertEquals(TermsEnum.SeekStatus.NOT_FOUND, result); + } + } + + if (loc >= 0) { + assertEquals(t, te.term()); + } else if (doSeekExact) { + // TermsEnum is unpositioned if seekExact returns false + continue; + } else if (loc == END_LOC) { + continue; + } else { + loc = -loc-1; + assertEquals(validTerms[loc], te.term()); + } + + // Do a bunch of next's after the seek + final int numNext = random.nextInt(validTerms.length); + + for(int nextCount=0;nextCount pair : pairs) { if (pair.output instanceof UpToTwoPositiveIntOutputs.TwoLongs) { @@ -872,15 +880,15 @@ } } - //System.out.println("TEST: after prune"); - /* - for(Map.Entry ent : prefixes.entrySet()) { - System.out.println(" " + inputToString(inputMode, ent.getKey()) + ": isLeaf=" + ent.getValue().isLeaf + " isFinal=" + ent.getValue().isFinal); - if (ent.getValue().isFinal) { - System.out.println(" finalOutput=" + outputs.outputToString(ent.getValue().finalOutput)); + if (VERBOSE) { + System.out.println("TEST: after prune"); + for(Map.Entry> ent : prefixes.entrySet()) { + System.out.println(" " + inputToString(inputMode, ent.getKey()) + ": isLeaf=" + ent.getValue().isLeaf + " isFinal=" + ent.getValue().isFinal); + if (ent.getValue().isFinal) { + System.out.println(" finalOutput=" + outputs.outputToString(ent.getValue().finalOutput)); + } } - } - */ + } if (prefixes.size() <= 1) { assertNull(fst); @@ -1019,7 +1027,10 @@ final long stopTime = System.currentTimeMillis() + RUN_TIME_MSEC; Document doc; int docCount = 0; - while((doc = docs.nextDoc()) != null && System.currentTimeMillis() < stopTime) { + final int numDocs = atLeast(400); + // nocommit + //while((doc = docs.nextDoc()) != null && System.currentTimeMillis() < stopTime) { + while((doc = docs.nextDoc()) != null && docCount < numDocs) { writer.addDocument(doc); docCount++; } @@ -1081,7 +1092,7 @@ final BytesRef randomTerm = new BytesRef(getRandomString()); if (VERBOSE) { - System.out.println("TEST: seek " + randomTerm.utf8ToString() + " " + randomTerm); + System.out.println("TEST: seek non-exist " + randomTerm.utf8ToString() + " " + randomTerm); } final TermsEnum.SeekStatus seekResult = termsEnum.seekCeil(randomTerm); @@ -1133,10 +1144,10 @@ assertEquals(termsEnum.term().utf8ToString() + " != " + fstEnum.current().input.utf8ToString(), termsEnum.term(), fstEnum.current().input); if (storeOrd) { // fst stored the ord - assertEquals(termsEnum.ord(), ((Long) fstEnum.current().output).longValue()); + assertEquals("term=" + termsEnum.term().utf8ToString() + " " + termsEnum.term(), termsEnum.ord(), ((Long) fstEnum.current().output).longValue()); } else { // fst stored the docFreq - assertEquals(termsEnum.docFreq(), (int) (((Long) fstEnum.current().output).longValue())); + assertEquals("term=" + termsEnum.term().utf8ToString() + " " + termsEnum.term(), termsEnum.docFreq(), (int) (((Long) fstEnum.current().output).longValue())); } } } @@ -1154,7 +1165,7 @@ this.inputMode = inputMode; this.outputs = outputs; - builder = new Builder(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs); + builder = new Builder(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null); } protected abstract T getOutput(IntsRef input, int ord) throws IOException; @@ -1254,7 +1265,7 @@ } } - // java -cp build/classes/test:build/classes/java:build/classes/test-framework:lib/junit-4.7.jar org.apache.lucene.util.fst.TestFSTs /x/tmp/allTerms3.txt out + // java -cp build/classes/test:build/classes/test-framework:build/classes/java:lib/junit-4.7.jar org.apache.lucene.util.automaton.fst.TestFSTs /x/tmp/allTerms3.txt out public static void main(String[] args) throws IOException { int prune = 0; int limit = Integer.MAX_VALUE; @@ -1411,6 +1422,202 @@ assertEquals(42, (long) seekResult.output); } + public void testPrimaryKeys() throws Exception { + Directory dir = newDirectory(); + + for(int cycle=0;cycle<2;cycle++) { + if (VERBOSE) { + System.out.println("TEST: cycle=" + cycle); + } + RandomIndexWriter w = new RandomIndexWriter(random, dir, + newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(IndexWriterConfig.OpenMode.CREATE)); + Document doc = new Document(); + Field idField = newField("id", "", Field.Index.NOT_ANALYZED); + doc.add(idField); + + final long seed = random.nextLong(); + + final int NUM_IDS = (int) (1000*RANDOM_MULTIPLIER*(1.0+random.nextDouble())); + //final int NUM_IDS = (int) (377 * (1.0+random.nextDouble())); + if (VERBOSE) { + System.out.println("TEST: NUM_IDS=" + NUM_IDS); + } + final Set allIDs = new HashSet(); + for(int id=0;id allIDsList = new ArrayList(allIDs); + final List sortedAllIDsList = new ArrayList(allIDsList); + Collections.sort(sortedAllIDsList); + + // Sprinkle in some non-existent PKs: + Set outOfBounds = new HashSet(); + for(int idx=0;idx builder = new Builder(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null); + builder.add("stat", outputs.get(17)); + builder.add("station", outputs.get(10)); + final FST fst = builder.finish(); + //Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot")); + StringWriter w = new StringWriter(); + Util.toDot(fst, w, false, false); + w.close(); + assertTrue(w.toString().indexOf("label=\"t/[7]\"") != -1); + } + + public void testInternalFinalState() throws Exception { + final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true); + + final Builder builder = new Builder(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null); + builder.add(new BytesRef("stat"), outputs.getNoOutput()); + builder.add(new BytesRef("station"), outputs.getNoOutput()); + final FST fst = builder.finish(); + StringWriter w = new StringWriter(); + //Writer w = new OutputStreamWriter(new FileOutputStream("/x/tmp/out.dot")); + Util.toDot(fst, w, false, false); + w.close(); + assertTrue(w.toString().indexOf("6 [shape=doublecircle") != -1); + } + // Make sure raw FST can differentiate between final vs // non-final end nodes public void testNonFinalStopNodes() throws Exception { Index: modules/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java --- modules/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java Mon Aug 01 12:07:14 2011 -0400 +++ modules/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java Mon Aug 01 12:09:21 2011 -0400 @@ -400,7 +400,7 @@ if (terms == null) { return Collections.emptyList(); } - FuzzyTermsEnum e = new FuzzyTermsEnum(terms.iterator(), atts, term, editDistance, Math.max(minPrefix, editDistance-1)); + FuzzyTermsEnum e = new FuzzyTermsEnum(terms, atts, term, editDistance, Math.max(minPrefix, editDistance-1)); final PriorityQueue stQueue = new PriorityQueue(); BytesRef queryTerm = new BytesRef(term.text());