LUCENE-5969: Copy block tree to backward codecs for 4.0-4.10, remove conditionals in 50 version, add getStats() TermsEnum API to remove need to expose Stats impl

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1633429 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Ryan Ernst 2014-10-21 18:15:32 +00:00
parent 03a6d8ce3c
commit 03c67fece3
18 changed files with 4427 additions and 70 deletions

View File

@ -0,0 +1,326 @@
package org.apache.lucene.codecs.blocktree;
/*
* 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.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.TreeMap;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/** A block-based terms index and dictionary that assigns
* terms to variable length blocks according to how they
* share prefixes. The terms index is a prefix trie
* whose leaves are term blocks. The advantage of this
* approach is that seekExact is often able to
* determine a term cannot exist without doing any IO, and
* intersection with Automata is very fast. Note that this
* terms dictionary has it's own fixed terms index (ie, it
* does not support a pluggable terms index
* implementation).
*
* <p><b>NOTE</b>: this terms dictionary supports
* min/maxItemsPerBlock during indexing to control how
* much memory the terms index uses.</p>
*
* <p>The data structure used by this implementation is very
* similar to a burst trie
* (http://citeseer.ist.psu.edu/viewdoc/summary?doi=10.1.1.18.3499),
* but with added logic to break up too-large blocks of all
* terms sharing a given prefix into smaller ones.</p>
*
* <p>Use {@link org.apache.lucene.index.CheckIndex} with the <code>-verbose</code>
* option to see summary statistics on the blocks in the
* dictionary.
*
* See {@link Lucene40BlockTreeTermsWriter}.
*
* @lucene.experimental
* @deprecated Only for 4.x backcompat
*/
@Deprecated
public final class Lucene40BlockTreeTermsReader extends FieldsProducer {
// Open input to the main terms dict file (_X.tib)
final IndexInput in;
//private static final boolean DEBUG = BlockTreeTermsWriter.DEBUG;
// Reads the terms dict entries, to gather state to
// produce DocsEnum on demand
final PostingsReaderBase postingsReader;
private final TreeMap<String,Lucene40FieldReader> fields = new TreeMap<>();
/** File offset where the directory starts in the terms file. */
private long dirOffset;
/** File offset where the directory starts in the index file. */
private long indexDirOffset;
final String segment;
private final int version;
/** Sole constructor. */
public Lucene40BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState state)
throws IOException {
this.postingsReader = postingsReader;
this.segment = state.segmentInfo.name;
String termsFileName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, Lucene40BlockTreeTermsWriter.TERMS_EXTENSION);
in = state.directory.openInput(termsFileName, state.context);
boolean success = false;
IndexInput indexIn = null;
try {
version = readHeader(in);
String indexFileName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, Lucene40BlockTreeTermsWriter.TERMS_INDEX_EXTENSION);
indexIn = state.directory.openInput(indexFileName, state.context);
int indexVersion = readIndexHeader(indexIn);
if (indexVersion != version) {
throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion, indexIn);
}
// verify
if (version >= Lucene40BlockTreeTermsWriter.VERSION_CHECKSUM) {
CodecUtil.checksumEntireFile(indexIn);
}
// Have PostingsReader init itself
postingsReader.init(in, state);
// NOTE: data file is too costly to verify checksum against all the bytes on open,
// but for now we at least verify proper structure of the checksum footer: which looks
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
// such as file truncation.
if (version >= Lucene40BlockTreeTermsWriter.VERSION_CHECKSUM) {
CodecUtil.retrieveChecksum(in);
}
// Read per-field details
seekDir(in, dirOffset);
seekDir(indexIn, indexDirOffset);
final int numFields = in.readVInt();
if (numFields < 0) {
throw new CorruptIndexException("invalid numFields: " + numFields, in);
}
for(int i=0;i<numFields;i++) {
final int field = in.readVInt();
final long numTerms = in.readVLong();
if (numTerms <= 0) {
throw new CorruptIndexException("Illegal numTerms for field number: " + field, in);
}
final int numBytes = in.readVInt();
if (numBytes < 0) {
throw new CorruptIndexException("invalid rootCode for field number: " + field + ", numBytes=" + numBytes, in);
}
final BytesRef rootCode = new BytesRef(new byte[numBytes]);
in.readBytes(rootCode.bytes, 0, numBytes);
rootCode.length = numBytes;
final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
if (fieldInfo == null) {
throw new CorruptIndexException("invalid field number: " + field, in);
}
final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
final long sumDocFreq = in.readVLong();
final int docCount = in.readVInt();
final int longsSize = version >= Lucene40BlockTreeTermsWriter.VERSION_META_ARRAY ? in.readVInt() : 0;
if (longsSize < 0) {
throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, in);
}
BytesRef minTerm, maxTerm;
if (version >= Lucene40BlockTreeTermsWriter.VERSION_MIN_MAX_TERMS) {
minTerm = readBytesRef(in);
maxTerm = readBytesRef(in);
} else {
minTerm = maxTerm = null;
}
if (docCount < 0 || docCount > state.segmentInfo.getDocCount()) { // #docs with field must be <= #docs
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.getDocCount(), in);
}
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, in);
}
if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq, in);
}
final long indexStartFP = indexIn.readVLong();
Lucene40FieldReader previous = fields.put(fieldInfo.name,
new Lucene40FieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount,
indexStartFP, longsSize, indexIn, minTerm, maxTerm));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name, in);
}
}
indexIn.close();
success = true;
} finally {
if (!success) {
// this.close() will close in:
IOUtils.closeWhileHandlingException(indexIn, this);
}
}
}
private static BytesRef readBytesRef(IndexInput in) throws IOException {
BytesRef bytes = new BytesRef();
bytes.length = in.readVInt();
bytes.bytes = new byte[bytes.length];
in.readBytes(bytes.bytes, 0, bytes.length);
return bytes;
}
/** Reads terms file header. */
private int readHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, Lucene40BlockTreeTermsWriter.TERMS_CODEC_NAME,
Lucene40BlockTreeTermsWriter.VERSION_START,
Lucene40BlockTreeTermsWriter.VERSION_CURRENT);
if (version < Lucene40BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
dirOffset = input.readLong();
}
return version;
}
/** Reads index file header. */
private int readIndexHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, Lucene40BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
Lucene40BlockTreeTermsWriter.VERSION_START,
Lucene40BlockTreeTermsWriter.VERSION_CURRENT);
if (version < Lucene40BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
indexDirOffset = input.readLong();
}
return version;
}
/** Seek {@code input} to the directory offset. */
private void seekDir(IndexInput input, long dirOffset)
throws IOException {
if (version >= Lucene40BlockTreeTermsWriter.VERSION_CHECKSUM) {
input.seek(input.length() - CodecUtil.footerLength() - 8);
dirOffset = input.readLong();
} else if (version >= Lucene40BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
input.seek(input.length() - 8);
dirOffset = input.readLong();
}
input.seek(dirOffset);
}
// for debugging
// private static String toHex(int v) {
// return "0x" + Integer.toHexString(v);
// }
@Override
public void close() throws IOException {
try {
IOUtils.close(in, postingsReader);
} finally {
// Clear so refs to terms index is GCable even if
// app hangs onto us:
fields.clear();
}
}
@Override
public Iterator<String> iterator() {
return Collections.unmodifiableSet(fields.keySet()).iterator();
}
@Override
public Terms terms(String field) throws IOException {
assert field != null;
return fields.get(field);
}
@Override
public int size() {
return fields.size();
}
// for debugging
String brToString(BytesRef b) {
if (b == null) {
return "null";
} else {
try {
return b.utf8ToString() + " " + b;
} catch (Throwable t) {
// If BytesRef isn't actually UTF8, or it's eg a
// prefix of UTF8 that ends mid-unicode-char, we
// fallback to hex:
return b.toString();
}
}
}
@Override
public long ramBytesUsed() {
long sizeInBytes = postingsReader.ramBytesUsed();
for(Lucene40FieldReader reader : fields.values()) {
sizeInBytes += reader.ramBytesUsed();
}
return sizeInBytes;
}
@Override
public Iterable<? extends Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.addAll(Accountables.namedAccountables("field", fields));
resources.add(Accountables.namedAccountable("delegate", postingsReader));
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
if (version >= Lucene40BlockTreeTermsWriter.VERSION_CHECKSUM) {
// term dictionary
CodecUtil.checksumEntireFile(in);
// postings
postingsReader.checkIntegrity();
}
}
@Override
public String toString() {
return getClass().getSimpleName() + "(fields=" + fields.size() + ",delegate=" + postingsReader + ")";
}
}

View File

@ -0,0 +1,202 @@
package org.apache.lucene.codecs.blocktree;
/*
* 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.Collections;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
/**
* BlockTree's implementation of {@link Terms}.
* @deprecated Only for 4.x backcompat
*/
@Deprecated
final class Lucene40FieldReader extends Terms implements Accountable {
private static final long BASE_RAM_BYTES_USED =
RamUsageEstimator.shallowSizeOfInstance(Lucene40FieldReader.class)
+ 3 * RamUsageEstimator.shallowSizeOfInstance(BytesRef.class);
final long numTerms;
final FieldInfo fieldInfo;
final long sumTotalTermFreq;
final long sumDocFreq;
final int docCount;
final long indexStartFP;
final long rootBlockFP;
final BytesRef rootCode;
final BytesRef minTerm;
final BytesRef maxTerm;
final int longsSize;
final Lucene40BlockTreeTermsReader parent;
final FST<BytesRef> index;
//private boolean DEBUG;
Lucene40FieldReader(Lucene40BlockTreeTermsReader parent, FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount,
long indexStartFP, int longsSize, IndexInput indexIn, BytesRef minTerm, BytesRef maxTerm) throws IOException {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
//DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
this.parent = parent;
this.numTerms = numTerms;
this.sumTotalTermFreq = sumTotalTermFreq;
this.sumDocFreq = sumDocFreq;
this.docCount = docCount;
this.indexStartFP = indexStartFP;
this.rootCode = rootCode;
this.longsSize = longsSize;
this.minTerm = minTerm;
this.maxTerm = maxTerm;
// 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() >>> Lucene40BlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
if (indexIn != null) {
final IndexInput clone = 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();
}
*/
} else {
index = null;
}
}
@Override
public BytesRef getMin() throws IOException {
if (minTerm == null) {
// Older index that didn't store min/maxTerm
return super.getMin();
} else {
return minTerm;
}
}
@Override
public BytesRef getMax() throws IOException {
if (maxTerm == null) {
// Older index that didn't store min/maxTerm
return super.getMax();
} else {
return maxTerm;
}
}
/** For debugging -- used by CheckIndex too*/
// TODO: maybe push this into Terms?
public Lucene40Stats computeStats() throws IOException {
return new Lucene40SegmentTermsEnum(this).computeBlockStats();
}
@Override
public boolean hasFreqs() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
}
@Override
public boolean hasOffsets() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
}
@Override
public boolean hasPositions() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
}
@Override
public boolean hasPayloads() {
return fieldInfo.hasPayloads();
}
@Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
return new Lucene40SegmentTermsEnum(this);
}
@Override
public long size() {
return numTerms;
}
@Override
public long getSumTotalTermFreq() {
return sumTotalTermFreq;
}
@Override
public long getSumDocFreq() {
return sumDocFreq;
}
@Override
public int getDocCount() {
return docCount;
}
@Override
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
if (compiled.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
}
return new Lucene40IntersectTermsEnum(this, compiled, startTerm);
}
@Override
public long ramBytesUsed() {
return BASE_RAM_BYTES_USED + ((index!=null)? index.ramBytesUsed() : 0);
}
@Override
public Iterable<? extends Accountable> getChildResources() {
if (index == null) {
return Collections.emptyList();
} else {
return Collections.singleton(Accountables.namedAccountable("term index", index));
}
}
@Override
public String toString() {
return "BlockTreeTerms(terms=" + numTerms + ",postings=" + sumDocFreq + ",positions=" + sumTotalTermFreq + ",docs=" + docCount + ")";
}
}

View File

@ -0,0 +1,490 @@
package org.apache.lucene.codecs.blocktree;
/*
* 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.IndexOptions;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.TermsEnum;
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.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RunAutomaton;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Outputs;
// NOTE: cannot seek!
/**
* @deprecated Only for 4.x backcompat
*/
@Deprecated
final class Lucene40IntersectTermsEnum extends TermsEnum {
final IndexInput in;
final static Outputs<BytesRef> fstOutputs = ByteSequenceOutputs.getSingleton();
private Lucene40IntersectTermsEnumFrame[] stack;
@SuppressWarnings({"rawtypes","unchecked"}) private FST.Arc<BytesRef>[] arcs = new FST.Arc[5];
final RunAutomaton runAutomaton;
final CompiledAutomaton compiledAutomaton;
private Lucene40IntersectTermsEnumFrame currentFrame;
private final BytesRef term = new BytesRef();
private final FST.BytesReader fstReader;
final Lucene40FieldReader fr;
private BytesRef savedStartTerm;
// TODO: in some cases we can filter by length? eg
// regexp foo*bar must be at least length 6 bytes
public Lucene40IntersectTermsEnum(Lucene40FieldReader fr, CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
// if (DEBUG) {
// System.out.println("\nintEnum.init seg=" + segment + " commonSuffix=" + brToString(compiled.commonSuffixRef));
// }
this.fr = fr;
runAutomaton = compiled.runAutomaton;
compiledAutomaton = compiled;
in = fr.parent.in.clone();
stack = new Lucene40IntersectTermsEnumFrame[5];
for(int idx=0;idx<stack.length;idx++) {
stack[idx] = new Lucene40IntersectTermsEnumFrame(this, idx);
}
for(int arcIdx=0;arcIdx<arcs.length;arcIdx++) {
arcs[arcIdx] = new FST.Arc<>();
}
if (fr.index == null) {
fstReader = null;
} else {
fstReader = fr.index.getBytesReader();
}
// TODO: if the automaton is "smallish" we really
// should use the terms index to seek at least to
// the initial term and likely to subsequent terms
// (or, maybe just fallback to ATE for such cases).
// Else the seek cost of loading the frames will be
// too costly.
final FST.Arc<BytesRef> arc = fr.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 Lucene40IntersectTermsEnumFrame f = stack[0];
f.fp = f.fpOrig = fr.rootBlockFP;
f.prefix = 0;
f.setState(runAutomaton.getInitialState());
f.arc = arc;
f.outputPrefix = arc.output;
f.load(fr.rootCode);
// for assert:
assert setSavedStartTerm(startTerm);
currentFrame = f;
if (startTerm != null) {
seekToStartTerm(startTerm);
}
}
// only for assert:
private boolean setSavedStartTerm(BytesRef startTerm) {
savedStartTerm = startTerm == null ? null : BytesRef.deepCopyOf(startTerm);
return true;
}
@Override
public TermState termState() throws IOException {
currentFrame.decodeMetaData();
return currentFrame.termState.clone();
}
private Lucene40IntersectTermsEnumFrame getFrame(int ord) throws IOException {
if (ord >= stack.length) {
final Lucene40IntersectTermsEnumFrame[] next = new Lucene40IntersectTermsEnumFrame[ArrayUtil.oversize(1+ord, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
System.arraycopy(stack, 0, next, 0, stack.length);
for(int stackOrd=stack.length;stackOrd<next.length;stackOrd++) {
next[stackOrd] = new Lucene40IntersectTermsEnumFrame(this, stackOrd);
}
stack = next;
}
assert stack[ord].ord == ord;
return stack[ord];
}
private FST.Arc<BytesRef> getArc(int ord) {
if (ord >= arcs.length) {
@SuppressWarnings({"rawtypes","unchecked"}) final FST.Arc<BytesRef>[] 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<next.length;arcOrd++) {
next[arcOrd] = new FST.Arc<>();
}
arcs = next;
}
return arcs[ord];
}
private Lucene40IntersectTermsEnumFrame pushFrame(int state) throws IOException {
final Lucene40IntersectTermsEnumFrame f = getFrame(currentFrame == null ? 0 : 1+currentFrame.ord);
f.fp = f.fpOrig = currentFrame.lastSubFP;
f.prefix = currentFrame.prefix + currentFrame.suffix;
// if (DEBUG) System.out.println(" pushFrame state=" + state + " prefix=" + f.prefix);
f.setState(state);
// Walk the arc through the index -- we only
// "bother" with this so we can get the floor data
// from the index and skip floor blocks when
// possible:
FST.Arc<BytesRef> arc = currentFrame.arc;
int idx = currentFrame.prefix;
assert currentFrame.suffix > 0;
BytesRef output = currentFrame.outputPrefix;
while (idx < f.prefix) {
final int target = term.bytes[idx] & 0xff;
// TODO: we could be more efficient for the next()
// case by using current arc as starting point,
// passed to findTargetArc
arc = fr.index.findTargetArc(target, arc, getArc(1+idx), fstReader);
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() {
return term;
}
@Override
public int docFreq() throws IOException {
//if (DEBUG) System.out.println("BTIR.docFreq");
currentFrame.decodeMetaData();
//if (DEBUG) System.out.println(" return " + currentFrame.termState.docFreq);
return currentFrame.termState.docFreq;
}
@Override
public long totalTermFreq() throws IOException {
currentFrame.decodeMetaData();
return currentFrame.termState.totalTermFreq;
}
@Override
public DocsEnum docs(Bits skipDocs, DocsEnum reuse, int flags) throws IOException {
currentFrame.decodeMetaData();
return fr.parent.postingsReader.docs(fr.fieldInfo, currentFrame.termState, skipDocs, reuse, flags);
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
if (fr.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
// Positions were not indexed:
return null;
}
currentFrame.decodeMetaData();
return fr.parent.postingsReader.docsAndPositions(fr.fieldInfo, currentFrame.termState, skipDocs, reuse, flags);
}
private int getState() {
int state = currentFrame.state;
for(int idx=0;idx<currentFrame.suffix;idx++) {
state = runAutomaton.step(state, currentFrame.suffixBytes[currentFrame.startBytePos+idx] & 0xff);
assert state != -1;
}
return state;
}
// NOTE: specialized to only doing the first-time
// seek, but we could generalize it to allow
// arbitrary seekExact/Ceil. Note that this is a
// seekFloor!
private void seekToStartTerm(BytesRef target) throws IOException {
//if (DEBUG) 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<BytesRef> arc = arcs[0];
assert arc == currentFrame.arc;
for(int idx=0;idx<=target.length;idx++) {
while (true) {
final int savePos = currentFrame.suffixesReader.getPosition();
final int saveStartBytePos = currentFrame.startBytePos;
final int saveSuffix = currentFrame.suffix;
final long saveLastSubFP = currentFrame.lastSubFP;
final int saveTermBlockOrd = currentFrame.termState.termBlockOrd;
final boolean isSubBlock = currentFrame.next();
//if (DEBUG) System.out.println(" cycle ent=" + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") prefix=" + currentFrame.prefix + " suffix=" + currentFrame.suffix + " isBlock=" + isSubBlock + " firstLabel=" + (currentFrame.suffix == 0 ? "" : (currentFrame.suffixBytes[currentFrame.startBytePos])&0xff));
term.length = currentFrame.prefix + currentFrame.suffix;
if (term.bytes.length < term.length) {
term.bytes = ArrayUtil.grow(term.bytes, term.length);
}
System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
if (isSubBlock && StringHelper.startsWith(target, term)) {
// Recurse
//if (DEBUG) System.out.println(" recurse!");
currentFrame = pushFrame(getState());
break;
} else {
final int cmp = term.compareTo(target);
if (cmp < 0) {
if (currentFrame.nextEnt == currentFrame.entCount) {
if (!currentFrame.isLastInFloor) {
//if (DEBUG) System.out.println(" load floorBlock");
currentFrame.loadNextFloorBlock();
continue;
} else {
//if (DEBUG) System.out.println(" return term=" + brToString(term));
return;
}
}
continue;
} else if (cmp == 0) {
//if (DEBUG) System.out.println(" return term=" + brToString(term));
return;
} else {
// Fallback to prior entry: the semantics of
// this method is that the first call to
// next() will return the term after the
// requested term
currentFrame.nextEnt--;
currentFrame.lastSubFP = saveLastSubFP;
currentFrame.startBytePos = saveStartBytePos;
currentFrame.suffix = saveSuffix;
currentFrame.suffixesReader.setPosition(savePos);
currentFrame.termState.termBlockOrd = saveTermBlockOrd;
System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
term.length = currentFrame.prefix + currentFrame.suffix;
// If the last entry was a block we don't
// need to bother recursing and pushing to
// the last term under it because the first
// next() will simply skip the frame anyway
return;
}
}
}
}
assert false;
}
@Override
public BytesRef next() throws IOException {
// if (DEBUG) {
// System.out.println("\nintEnum.next seg=" + segment);
// System.out.println(" frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
// }
nextTerm:
while(true) {
// Pop finished frames
while (currentFrame.nextEnt == currentFrame.entCount) {
if (!currentFrame.isLastInFloor) {
//if (DEBUG) System.out.println(" next-floor-block");
currentFrame.loadNextFloorBlock();
//if (DEBUG) System.out.println("\n frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
} else {
//if (DEBUG) System.out.println(" pop frame");
if (currentFrame.ord == 0) {
return null;
}
final long lastFP = currentFrame.fpOrig;
currentFrame = stack[currentFrame.ord-1];
assert currentFrame.lastSubFP == lastFP;
//if (DEBUG) System.out.println("\n frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
}
}
final boolean isSubBlock = currentFrame.next();
// if (DEBUG) {
// final BytesRef suffixRef = new BytesRef();
// suffixRef.bytes = currentFrame.suffixBytes;
// suffixRef.offset = currentFrame.startBytePos;
// suffixRef.length = currentFrame.suffix;
// System.out.println(" " + (isSubBlock ? "sub-block" : "term") + " " + currentFrame.nextEnt + " (of " + currentFrame.entCount + ") suffix=" + brToString(suffixRef));
// }
if (currentFrame.suffix != 0) {
final int label = currentFrame.suffixBytes[currentFrame.startBytePos] & 0xff;
while (label > currentFrame.curTransitionMax) {
if (currentFrame.transitionIndex >= currentFrame.transitionCount-1) {
// Stop processing this frame -- no further
// matches are possible because we've moved
// beyond what the max transition will allow
//if (DEBUG) System.out.println(" break: trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]));
// sneaky! forces a pop above
currentFrame.isLastInFloor = true;
currentFrame.nextEnt = currentFrame.entCount;
continue nextTerm;
}
currentFrame.transitionIndex++;
compiledAutomaton.automaton.getNextTransition(currentFrame.transition);
currentFrame.curTransitionMax = currentFrame.transition.max;
//if (DEBUG) System.out.println(" next trans=" + currentFrame.transitions[currentFrame.transitionIndex]);
}
}
// First test the common suffix, if set:
if (compiledAutomaton.commonSuffixRef != null && !isSubBlock) {
final int termLen = currentFrame.prefix + currentFrame.suffix;
if (termLen < compiledAutomaton.commonSuffixRef.length) {
// No match
// if (DEBUG) {
// System.out.println(" skip: common suffix length");
// }
continue nextTerm;
}
final byte[] suffixBytes = currentFrame.suffixBytes;
final byte[] commonSuffixBytes = compiledAutomaton.commonSuffixRef.bytes;
final int lenInPrefix = compiledAutomaton.commonSuffixRef.length - currentFrame.suffix;
assert compiledAutomaton.commonSuffixRef.offset == 0;
int suffixBytesPos;
int commonSuffixBytesPos = 0;
if (lenInPrefix > 0) {
// A prefix of the common suffix overlaps with
// the suffix of the block prefix so we first
// test whether the prefix part matches:
final byte[] termBytes = term.bytes;
int termBytesPos = currentFrame.prefix - lenInPrefix;
assert termBytesPos >= 0;
final int termBytesPosEnd = currentFrame.prefix;
while (termBytesPos < termBytesPosEnd) {
if (termBytes[termBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
// if (DEBUG) {
// System.out.println(" skip: common suffix mismatch (in prefix)");
// }
continue nextTerm;
}
}
suffixBytesPos = currentFrame.startBytePos;
} else {
suffixBytesPos = currentFrame.startBytePos + currentFrame.suffix - compiledAutomaton.commonSuffixRef.length;
}
// Test overlapping suffix part:
final int commonSuffixBytesPosEnd = compiledAutomaton.commonSuffixRef.length;
while (commonSuffixBytesPos < commonSuffixBytesPosEnd) {
if (suffixBytes[suffixBytesPos++] != commonSuffixBytes[commonSuffixBytesPos++]) {
// if (DEBUG) {
// System.out.println(" skip: common suffix mismatch");
// }
continue nextTerm;
}
}
}
// TODO: maybe we should do the same linear test
// that AutomatonTermsEnum does, so that if we
// reach a part of the automaton where .* is
// "temporarily" accepted, we just blindly .next()
// until the limit
// See if the term prefix matches the automaton:
int state = currentFrame.state;
for (int idx=0;idx<currentFrame.suffix;idx++) {
state = runAutomaton.step(state, currentFrame.suffixBytes[currentFrame.startBytePos+idx] & 0xff);
if (state == -1) {
// No match
//System.out.println(" no s=" + state);
continue nextTerm;
} else {
//System.out.println(" c s=" + state);
}
}
if (isSubBlock) {
// Match! Recurse:
//if (DEBUG) System.out.println(" sub-block match to state=" + state + "; recurse fp=" + currentFrame.lastSubFP);
copyTerm();
currentFrame = pushFrame(state);
//if (DEBUG) System.out.println("\n frame ord=" + currentFrame.ord + " prefix=" + brToString(new BytesRef(term.bytes, term.offset, currentFrame.prefix)) + " state=" + currentFrame.state + " lastInFloor?=" + currentFrame.isLastInFloor + " fp=" + currentFrame.fp + " trans=" + (currentFrame.transitions.length == 0 ? "n/a" : currentFrame.transitions[currentFrame.transitionIndex]) + " outputPrefix=" + currentFrame.outputPrefix);
} else if (runAutomaton.isAccept(state)) {
copyTerm();
//if (DEBUG) System.out.println(" term match to state=" + state + "; return term=" + brToString(term));
assert savedStartTerm == null || term.compareTo(savedStartTerm) > 0: "saveStartTerm=" + savedStartTerm.utf8ToString() + " term=" + term.utf8ToString();
return term;
} else {
//System.out.println(" no s=" + state);
}
}
}
private void copyTerm() {
//System.out.println(" copyTerm cur.prefix=" + currentFrame.prefix + " cur.suffix=" + currentFrame.suffix + " first=" + (char) currentFrame.suffixBytes[currentFrame.startBytePos]);
final int len = currentFrame.prefix + currentFrame.suffix;
if (term.bytes.length < len) {
term.bytes = ArrayUtil.grow(term.bytes, len);
}
System.arraycopy(currentFrame.suffixBytes, currentFrame.startBytePos, term.bytes, currentFrame.prefix, currentFrame.suffix);
term.length = len;
}
@Override
public boolean seekExact(BytesRef text) {
throw new UnsupportedOperationException();
}
@Override
public void seekExact(long ord) {
throw new UnsupportedOperationException();
}
@Override
public long ord() {
throw new UnsupportedOperationException();
}
@Override
public SeekStatus seekCeil(BytesRef text) {
throw new UnsupportedOperationException();
}
}

View File

@ -0,0 +1,302 @@
package org.apache.lucene.codecs.blocktree;
/*
* 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.codecs.BlockTermState;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.automaton.Transition;
import org.apache.lucene.util.fst.FST;
/**
* @deprecated Only for 4.x backcompat
*/
// TODO: can we share this with the frame in STE?
@Deprecated
final class Lucene40IntersectTermsEnumFrame {
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 transition = new Transition();
int curTransitionMax;
int transitionIndex;
int transitionCount;
FST.Arc<BytesRef> arc;
final BlockTermState termState;
// metadata buffer, holding monotonic values
public long[] longs;
// metadata buffer, holding general values
public byte[] bytes;
ByteArrayDataInput bytesReader;
// Cumulative output so far
BytesRef outputPrefix;
int startBytePos;
int suffix;
private final Lucene40IntersectTermsEnum ite;
public Lucene40IntersectTermsEnumFrame(Lucene40IntersectTermsEnum ite, int ord) throws IOException {
this.ite = ite;
this.ord = ord;
this.termState = ite.fr.parent.postingsReader.newTermState();
this.termState.totalTermFreq = -1;
this.longs = new long[ite.fr.longsSize];
}
void loadNextFloorBlock() throws IOException {
assert numFollowFloorBlocks > 0;
//if (DEBUG) System.out.println(" loadNextFoorBlock trans=" + transitions[transitionIndex]);
do {
fp = fpOrig + (floorDataReader.readVLong() >>> 1);
numFollowFloorBlocks--;
// if (DEBUG) System.out.println(" skip floor block2! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[transitionIndex].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
if (numFollowFloorBlocks != 0) {
nextFloorLabel = floorDataReader.readByte() & 0xff;
} else {
nextFloorLabel = 256;
}
// if (DEBUG) System.out.println(" nextFloorLabel=" + (char) nextFloorLabel);
} while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min);
load(null);
}
public void setState(int state) {
this.state = state;
transitionIndex = 0;
transitionCount = ite.compiledAutomaton.automaton.getNumTransitions(state);
if (transitionCount != 0) {
ite.compiledAutomaton.automaton.initTransition(state, transition);
ite.compiledAutomaton.automaton.getNextTransition(transition);
curTransitionMax = transition.max;
} else {
curTransitionMax = -1;
}
}
void load(BytesRef frameIndexData) throws IOException {
// if (DEBUG) System.out.println(" load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state));
if (frameIndexData != null && transitionCount != 0) {
// Floor frame
if (floorData.length < frameIndexData.length) {
this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
}
System.arraycopy(frameIndexData.bytes, frameIndexData.offset, floorData, 0, frameIndexData.length);
floorDataReader.reset(floorData, 0, frameIndexData.length);
// Skip first long -- has redundant fp, hasTerms
// flag, isFloor flag
final long code = floorDataReader.readVLong();
if ((code & Lucene40BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) {
numFollowFloorBlocks = floorDataReader.readVInt();
nextFloorLabel = floorDataReader.readByte() & 0xff;
// if (DEBUG) System.out.println(" numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
// If current state is accept, we must process
// first block in case it has empty suffix:
if (!ite.runAutomaton.isAccept(state)) {
// Maybe skip floor blocks:
assert transitionIndex == 0: "transitionIndex=" + transitionIndex;
while (numFollowFloorBlocks != 0 && nextFloorLabel <= transition.min) {
fp = fpOrig + (floorDataReader.readVLong() >>> 1);
numFollowFloorBlocks--;
// if (DEBUG) System.out.println(" skip floor block! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
if (numFollowFloorBlocks != 0) {
nextFloorLabel = floorDataReader.readByte() & 0xff;
} else {
nextFloorLabel = 256;
}
}
}
}
}
ite.in.seek(fp);
int code = ite.in.readVInt();
entCount = code >>> 1;
assert entCount > 0;
isLastInFloor = (code & 1) != 0;
// term suffixes:
code = ite.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)];
}
ite.in.readBytes(suffixBytes, 0, numBytes);
suffixesReader.reset(suffixBytes, 0, numBytes);
// stats
numBytes = ite.in.readVInt();
if (statBytes.length < numBytes) {
statBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
}
ite.in.readBytes(statBytes, 0, numBytes);
statsReader.reset(statBytes, 0, numBytes);
metaDataUpto = 0;
termState.termBlockOrd = 0;
nextEnt = 0;
// metadata
numBytes = ite.in.readVInt();
if (bytes == null) {
bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
bytesReader = new ByteArrayDataInput();
} else if (bytes.length < numBytes) {
bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
}
ite.in.readBytes(bytes, 0, numBytes);
bytesReader.reset(bytes, 0, numBytes);
if (!isLastInFloor) {
// Sub-blocks of a single floor block are always
// written one after another -- tail recurse:
fpEnd = ite.in.getFilePointer();
}
}
// TODO: maybe add scanToLabel; should give perf boost
public boolean next() {
return isLeafBlock ? nextLeaf() : nextNonLeaf();
}
// Decodes next entry; returns true if it's a sub-block
public boolean nextLeaf() {
//if (DEBUG) System.out.println(" frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
suffix = suffixesReader.readVInt();
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
return false;
}
public boolean nextNonLeaf() {
//if (DEBUG) System.out.println(" frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
final int code = suffixesReader.readVInt();
suffix = code >>> 1;
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
if ((code & 1) == 0) {
// A normal term
termState.termBlockOrd++;
return false;
} else {
// A sub-block; make sub-FP absolute:
lastSubFP = fp - suffixesReader.readVLong();
return true;
}
}
public int getTermBlockOrd() {
return isLeafBlock ? nextEnt : termState.termBlockOrd;
}
public void decodeMetaData() throws IOException {
// lazily catch up on metadata decode:
final int limit = getTermBlockOrd();
boolean absolute = metaDataUpto == 0;
assert limit > 0;
// 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:
// stats
termState.docFreq = statsReader.readVInt();
//if (DEBUG) System.out.println(" dF=" + state.docFreq);
if (ite.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
//if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq);
}
// metadata
for (int i = 0; i < ite.fr.longsSize; i++) {
longs[i] = bytesReader.readVLong();
}
ite.fr.parent.postingsReader.decodeTerm(longs, bytesReader, ite.fr.fieldInfo, termState, absolute);
metaDataUpto++;
absolute = false;
}
termState.termBlockOrd = metaDataUpto;
}
}

View File

@ -0,0 +1,732 @@
package org.apache.lucene.codecs.blocktree;
/*
* 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.codecs.BlockTermState;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.fst.FST;
/**
* @deprecated Only for 4.x backcompat
*/
@Deprecated
final class Lucene40SegmentTermsEnumFrame {
// Our index in stack[]:
final int ord;
boolean hasTerms;
boolean hasTermsOrig;
boolean isFloor;
FST.Arc<BytesRef> arc;
// File pointer where this block was loaded from
long fp;
long fpOrig;
long fpEnd;
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, or -1 if the block
// isn't loaded yet
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;
long lastSubFP;
int nextFloorLabel;
int numFollowFloorBlocks;
// Next term to decode metaData; we decode metaData
// lazily so that scanning to find the matching term is
// fast and only if you find a match and app wants the
// stats or docs/positions enums, will we decode the
// metaData
int metaDataUpto;
final BlockTermState state;
// metadata buffer, holding monotonic values
public long[] longs;
// metadata buffer, holding general values
public byte[] bytes;
ByteArrayDataInput bytesReader;
private final Lucene40SegmentTermsEnum ste;
public Lucene40SegmentTermsEnumFrame(Lucene40SegmentTermsEnum ste, int ord) throws IOException {
this.ste = ste;
this.ord = ord;
this.state = ste.fr.parent.postingsReader.newTermState();
this.state.totalTermFreq = -1;
this.longs = new long[ste.fr.longsSize];
}
public void setFloorData(ByteArrayDataInput in, BytesRef source) {
final int numBytes = source.length - (in.getPosition() - source.offset);
if (numBytes > floorData.length) {
floorData = new byte[ArrayUtil.oversize(numBytes, 1)];
}
System.arraycopy(source.bytes, source.offset+in.getPosition(), floorData, 0, numBytes);
floorDataReader.reset(floorData, 0, numBytes);
numFollowFloorBlocks = floorDataReader.readVInt();
nextFloorLabel = floorDataReader.readByte() & 0xff;
//if (DEBUG) {
//System.out.println(" setFloorData fpOrig=" + fpOrig + " bytes=" + new BytesRef(source.bytes, source.offset + in.getPosition(), numBytes) + " numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + toHex(nextFloorLabel));
//}
}
public int getTermBlockOrd() {
return isLeafBlock ? nextEnt : state.termBlockOrd;
}
void loadNextFloorBlock() throws IOException {
//if (DEBUG) {
//System.out.println(" loadNextFloorBlock fp=" + fp + " fpEnd=" + fpEnd);
//}
assert arc == null || isFloor: "arc=" + arc + " isFloor=" + isFloor;
fp = fpEnd;
nextEnt = -1;
loadBlock();
}
/* Does initial decode of next block of terms; this
doesn't actually decode the docFreq, totalTermFreq,
postings details (frq/prx offset, etc.) metadata;
it just loads them as byte[] blobs which are then
decoded on-demand if the metadata is ever requested
for any term in this block. This enables terms-only
intensive consumes (eg certain MTQs, respelling) to
not pay the price of decoding metadata they won't
use. */
void loadBlock() throws IOException {
// Clone the IndexInput lazily, so that consumers
// that just pull a TermsEnum to
// seekExact(TermState) don't pay this cost:
ste.initIndexInput();
if (nextEnt != -1) {
// Already loaded
return;
}
//System.out.println("blc=" + blockLoadCount);
ste.in.seek(fp);
int code = ste.in.readVInt();
entCount = code >>> 1;
assert entCount > 0;
isLastInFloor = (code & 1) != 0;
assert arc == null || (isLastInFloor || isFloor): "fp=" + fp + " arc=" + arc + " isFloor=" + isFloor + " isLastInFloor=" + isLastInFloor;
// TODO: if suffixes were stored in random-access
// array structure, then we could do binary search
// instead of linear scan to find target term; eg
// we could have simple array of offsets
// term suffixes:
code = ste.in.readVInt();
isLeafBlock = (code & 1) != 0;
int numBytes = code >>> 1;
if (suffixBytes.length < numBytes) {
suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
}
ste.in.readBytes(suffixBytes, 0, numBytes);
suffixesReader.reset(suffixBytes, 0, numBytes);
/*if (DEBUG) {
if (arc == null) {
System.out.println(" loadBlock (next) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
} else {
System.out.println(" loadBlock (seek) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
}
}*/
// stats
numBytes = ste.in.readVInt();
if (statBytes.length < numBytes) {
statBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
}
ste.in.readBytes(statBytes, 0, numBytes);
statsReader.reset(statBytes, 0, numBytes);
metaDataUpto = 0;
state.termBlockOrd = 0;
nextEnt = 0;
lastSubFP = -1;
// TODO: we could skip this if !hasTerms; but
// that's rare so won't help much
// metadata
numBytes = ste.in.readVInt();
if (bytes == null) {
bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
bytesReader = new ByteArrayDataInput();
} else if (bytes.length < numBytes) {
bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
}
ste.in.readBytes(bytes, 0, numBytes);
bytesReader.reset(bytes, 0, numBytes);
// Sub-blocks of a single floor block are always
// written one after another -- tail recurse:
fpEnd = ste.in.getFilePointer();
// if (DEBUG) {
// System.out.println(" fpEnd=" + fpEnd);
// }
}
void rewind() {
// Force reload:
fp = fpOrig;
nextEnt = -1;
hasTerms = hasTermsOrig;
if (isFloor) {
floorDataReader.rewind();
numFollowFloorBlocks = floorDataReader.readVInt();
assert numFollowFloorBlocks > 0;
nextFloorLabel = floorDataReader.readByte() & 0xff;
}
/*
//System.out.println("rewind");
// Keeps the block loaded, but rewinds its state:
if (nextEnt > 0 || fp != fpOrig) {
if (DEBUG) {
System.out.println(" rewind frame ord=" + ord + " fpOrig=" + fpOrig + " fp=" + fp + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " nextEnt=" + nextEnt + " prefixLen=" + prefix);
}
if (fp != fpOrig) {
fp = fpOrig;
nextEnt = -1;
} else {
nextEnt = 0;
}
hasTerms = hasTermsOrig;
if (isFloor) {
floorDataReader.rewind();
numFollowFloorBlocks = floorDataReader.readVInt();
nextFloorLabel = floorDataReader.readByte() & 0xff;
}
assert suffixBytes != null;
suffixesReader.rewind();
assert statBytes != null;
statsReader.rewind();
metaDataUpto = 0;
state.termBlockOrd = 0;
// TODO: skip this if !hasTerms? Then postings
// impl wouldn't have to write useless 0 byte
postingsReader.resetTermsBlock(fieldInfo, state);
lastSubFP = -1;
} else if (DEBUG) {
System.out.println(" skip rewind fp=" + fp + " fpOrig=" + fpOrig + " nextEnt=" + nextEnt + " ord=" + ord);
}
*/
}
public boolean next() {
return isLeafBlock ? nextLeaf() : nextNonLeaf();
}
// Decodes next entry; returns true if it's a sub-block
public boolean nextLeaf() {
//if (DEBUG) System.out.println(" frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
suffix = suffixesReader.readVInt();
startBytePos = suffixesReader.getPosition();
ste.term.setLength(prefix + suffix);
ste.term.grow(ste.term.length());
suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
// A normal term
ste.termExists = true;
return false;
}
public boolean nextNonLeaf() {
//if (DEBUG) System.out.println(" frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
nextEnt++;
final int code = suffixesReader.readVInt();
suffix = code >>> 1;
startBytePos = suffixesReader.getPosition();
ste.term.setLength(prefix + suffix);
ste.term.grow(ste.term.length());
suffixesReader.readBytes(ste.term.bytes(), prefix, suffix);
if ((code & 1) == 0) {
// A normal term
ste.termExists = true;
subCode = 0;
state.termBlockOrd++;
return false;
} else {
// A sub-block; make sub-FP absolute:
ste.termExists = false;
subCode = suffixesReader.readVLong();
lastSubFP = fp - subCode;
//if (DEBUG) {
//System.out.println(" lastSubFP=" + lastSubFP);
//}
return true;
}
}
// TODO: make this array'd so we can do bin search?
// likely not worth it? need to measure how many
// floor blocks we "typically" get
public void scanToFloorFrame(BytesRef target) {
if (!isFloor || target.length <= prefix) {
// if (DEBUG) {
// System.out.println(" scanToFloorFrame skip: isFloor=" + isFloor + " target.length=" + target.length + " vs prefix=" + prefix);
// }
return;
}
final int targetLabel = target.bytes[target.offset + prefix] & 0xFF;
// if (DEBUG) {
// System.out.println(" scanToFloorFrame fpOrig=" + fpOrig + " targetLabel=" + toHex(targetLabel) + " vs nextFloorLabel=" + toHex(nextFloorLabel) + " numFollowFloorBlocks=" + numFollowFloorBlocks);
// }
if (targetLabel < nextFloorLabel) {
// if (DEBUG) {
// System.out.println(" already on correct block");
// }
return;
}
assert numFollowFloorBlocks != 0;
long newFP = fpOrig;
while (true) {
final long code = floorDataReader.readVLong();
newFP = fpOrig + (code >>> 1);
hasTerms = (code & 1) != 0;
// if (DEBUG) {
// System.out.println(" label=" + toHex(nextFloorLabel) + " fp=" + newFP + " hasTerms?=" + hasTerms + " numFollowFloor=" + numFollowFloorBlocks);
// }
isLastInFloor = numFollowFloorBlocks == 1;
numFollowFloorBlocks--;
if (isLastInFloor) {
nextFloorLabel = 256;
// if (DEBUG) {
// System.out.println(" stop! last block nextFloorLabel=" + toHex(nextFloorLabel));
// }
break;
} else {
nextFloorLabel = floorDataReader.readByte() & 0xff;
if (targetLabel < nextFloorLabel) {
// if (DEBUG) {
// System.out.println(" stop! nextFloorLabel=" + toHex(nextFloorLabel));
// }
break;
}
}
}
if (newFP != fp) {
// Force re-load of the block:
// if (DEBUG) {
// System.out.println(" force switch to fp=" + newFP + " oldFP=" + fp);
// }
nextEnt = -1;
fp = newFP;
} else {
// if (DEBUG) {
// System.out.println(" stay on same fp=" + newFP);
// }
}
}
public void decodeMetaData() throws IOException {
//if (DEBUG) System.out.println("\nBTTR.decodeMetadata seg=" + segment + " mdUpto=" + metaDataUpto + " vs termBlockOrd=" + state.termBlockOrd);
// lazily catch up on metadata decode:
final int limit = getTermBlockOrd();
boolean absolute = metaDataUpto == 0;
assert limit > 0;
// 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:
// stats
state.docFreq = statsReader.readVInt();
//if (DEBUG) System.out.println(" dF=" + state.docFreq);
if (ste.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
state.totalTermFreq = state.docFreq + statsReader.readVLong();
//if (DEBUG) System.out.println(" totTF=" + state.totalTermFreq);
}
// metadata
for (int i = 0; i < ste.fr.longsSize; i++) {
longs[i] = bytesReader.readVLong();
}
ste.fr.parent.postingsReader.decodeTerm(longs, bytesReader, ste.fr.fieldInfo, state, absolute);
metaDataUpto++;
absolute = false;
}
state.termBlockOrd = metaDataUpto;
}
// Used only by assert
private boolean prefixMatches(BytesRef target) {
for(int bytePos=0;bytePos<prefix;bytePos++) {
if (target.bytes[target.offset + bytePos] != ste.term.byteAt(bytePos)) {
return false;
}
}
return true;
}
// Scans to sub-block that has this target fp; only
// called by next(); NOTE: does not set
// startBytePos/suffix as a side effect
public void scanToSubBlock(long subFP) {
assert !isLeafBlock;
//if (DEBUG) System.out.println(" scanToSubBlock fp=" + fp + " subFP=" + subFP + " entCount=" + entCount + " lastSubFP=" + lastSubFP);
//assert nextEnt == 0;
if (lastSubFP == subFP) {
//if (DEBUG) System.out.println(" already positioned");
return;
}
assert subFP < fp : "fp=" + fp + " subFP=" + subFP;
final long targetSubCode = fp - subFP;
//if (DEBUG) System.out.println(" targetSubCode=" + targetSubCode);
while(true) {
assert nextEnt < entCount;
nextEnt++;
final int code = suffixesReader.readVInt();
suffixesReader.skipBytes(isLeafBlock ? code : code >>> 1);
//if (DEBUG) System.out.println(" " + nextEnt + " (of " + entCount + ") ent isSubBlock=" + ((code&1)==1));
if ((code & 1) != 0) {
final long subCode = suffixesReader.readVLong();
//if (DEBUG) System.out.println(" subCode=" + subCode);
if (targetSubCode == subCode) {
//if (DEBUG) System.out.println(" match!");
lastSubFP = subFP;
return;
}
} else {
state.termBlockOrd++;
}
}
}
// NOTE: sets startBytePos/suffix as a side effect
public SeekStatus scanToTerm(BytesRef target, boolean exactOnly) throws IOException {
return isLeafBlock ? scanToTermLeaf(target, exactOnly) : scanToTermNonLeaf(target, exactOnly);
}
private int startBytePos;
private int suffix;
private long subCode;
// Target's prefix matches this block's prefix; we
// scan the entries check if the suffix matches.
public SeekStatus scanToTermLeaf(BytesRef target, boolean exactOnly) throws IOException {
// if (DEBUG) System.out.println(" scanToTermLeaf: block fp=" + fp + " prefix=" + prefix + " nextEnt=" + nextEnt + " (of " + entCount + ") target=" + brToString(target) + " term=" + brToString(term));
assert nextEnt != -1;
ste.termExists = true;
subCode = 0;
if (nextEnt == entCount) {
if (exactOnly) {
fillTerm();
}
return SeekStatus.END;
}
assert prefixMatches(target);
// Loop over each entry (term or sub-block) in this block:
//nextTerm: while(nextEnt < entCount) {
nextTerm: while (true) {
nextEnt++;
suffix = suffixesReader.readVInt();
// if (DEBUG) {
// BytesRef suffixBytesRef = new BytesRef();
// suffixBytesRef.bytes = suffixBytes;
// suffixBytesRef.offset = suffixesReader.getPosition();
// suffixBytesRef.length = suffix;
// System.out.println(" cycle: term " + (nextEnt-1) + " (of " + entCount + ") suffix=" + brToString(suffixBytesRef));
// }
final int termLen = prefix + suffix;
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
final int targetLimit = target.offset + (target.length < termLen ? target.length : termLen);
int targetPos = target.offset + prefix;
// Loop over bytes in the suffix, comparing to
// the target
int bytePos = startBytePos;
while(true) {
final int cmp;
final boolean stop;
if (targetPos < targetLimit) {
cmp = (suffixBytes[bytePos++]&0xFF) - (target.bytes[targetPos++]&0xFF);
stop = false;
} else {
assert targetPos == targetLimit;
cmp = termLen - target.length;
stop = true;
}
if (cmp < 0) {
// Current entry is still before the target;
// keep scanning
if (nextEnt == entCount) {
if (exactOnly) {
fillTerm();
}
// We are done scanning this block
break nextTerm;
} else {
continue nextTerm;
}
} else if (cmp > 0) {
// Done! Current entry is after target --
// return NOT_FOUND:
fillTerm();
//if (DEBUG) System.out.println(" not found");
return SeekStatus.NOT_FOUND;
} else if (stop) {
// Exact match!
// This cannot be a sub-block because we
// would have followed the index to this
// sub-block from the start:
assert ste.termExists;
fillTerm();
//if (DEBUG) System.out.println(" found!");
return SeekStatus.FOUND;
}
}
}
// It is possible (and OK) that terms index pointed us
// at this block, but, we scanned the entire block and
// did not find the term to position to. This happens
// when the target is after the last term in the block
// (but, before the next term in the index). EG
// target could be foozzz, and terms index pointed us
// to the foo* block, but the last term in this block
// was fooz (and, eg, first term in the next block will
// bee fop).
//if (DEBUG) System.out.println(" block end");
if (exactOnly) {
fillTerm();
}
// TODO: not consistent that in the
// not-exact case we don't next() into the next
// frame here
return SeekStatus.END;
}
// Target's prefix matches this block's prefix; we
// scan the entries check if the suffix matches.
public SeekStatus scanToTermNonLeaf(BytesRef target, boolean exactOnly) throws IOException {
//if (DEBUG) System.out.println(" scanToTermNonLeaf: block fp=" + fp + " prefix=" + prefix + " nextEnt=" + nextEnt + " (of " + entCount + ") target=" + brToString(target) + " term=" + brToString(term));
assert nextEnt != -1;
if (nextEnt == entCount) {
if (exactOnly) {
fillTerm();
ste.termExists = subCode == 0;
}
return SeekStatus.END;
}
assert prefixMatches(target);
// Loop over each entry (term or sub-block) in this block:
//nextTerm: while(nextEnt < entCount) {
nextTerm: while (true) {
nextEnt++;
final int code = suffixesReader.readVInt();
suffix = code >>> 1;
// if (DEBUG) {
// BytesRef suffixBytesRef = new BytesRef();
// suffixBytesRef.bytes = suffixBytes;
// suffixBytesRef.offset = suffixesReader.getPosition();
// suffixBytesRef.length = suffix;
// System.out.println(" cycle: " + ((code&1)==1 ? "sub-block" : "term") + " " + (nextEnt-1) + " (of " + entCount + ") suffix=" + brToString(suffixBytesRef));
// }
ste.termExists = (code & 1) == 0;
final int termLen = prefix + suffix;
startBytePos = suffixesReader.getPosition();
suffixesReader.skipBytes(suffix);
if (ste.termExists) {
state.termBlockOrd++;
subCode = 0;
} else {
subCode = suffixesReader.readVLong();
lastSubFP = fp - subCode;
}
final int targetLimit = target.offset + (target.length < termLen ? target.length : termLen);
int targetPos = target.offset + prefix;
// Loop over bytes in the suffix, comparing to
// the target
int bytePos = startBytePos;
while(true) {
final int cmp;
final boolean stop;
if (targetPos < targetLimit) {
cmp = (suffixBytes[bytePos++]&0xFF) - (target.bytes[targetPos++]&0xFF);
stop = false;
} else {
assert targetPos == targetLimit;
cmp = termLen - target.length;
stop = true;
}
if (cmp < 0) {
// Current entry is still before the target;
// keep scanning
if (nextEnt == entCount) {
if (exactOnly) {
fillTerm();
//termExists = true;
}
// We are done scanning this block
break nextTerm;
} else {
continue nextTerm;
}
} else if (cmp > 0) {
// Done! Current entry is after target --
// return NOT_FOUND:
fillTerm();
if (!exactOnly && !ste.termExists) {
// We are on a sub-block, and caller wants
// us to position to the next term after
// the target, so we must recurse into the
// sub-frame(s):
ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, termLen);
ste.currentFrame.loadBlock();
while (ste.currentFrame.next()) {
ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length());
ste.currentFrame.loadBlock();
}
}
//if (DEBUG) System.out.println(" not found");
return SeekStatus.NOT_FOUND;
} else if (stop) {
// Exact match!
// This cannot be a sub-block because we
// would have followed the index to this
// sub-block from the start:
assert ste.termExists;
fillTerm();
//if (DEBUG) System.out.println(" found!");
return SeekStatus.FOUND;
}
}
}
// It is possible (and OK) that terms index pointed us
// at this block, but, we scanned the entire block and
// did not find the term to position to. This happens
// when the target is after the last term in the block
// (but, before the next term in the index). EG
// target could be foozzz, and terms index pointed us
// to the foo* block, but the last term in this block
// was fooz (and, eg, first term in the next block will
// bee fop).
//if (DEBUG) System.out.println(" block end");
if (exactOnly) {
fillTerm();
}
// TODO: not consistent that in the
// not-exact case we don't next() into the next
// frame here
return SeekStatus.END;
}
private void fillTerm() {
final int termLength = prefix + suffix;
ste.term.setLength(termLength);
ste.term.grow(termLength);
System.arraycopy(suffixBytes, startBytePos, ste.term.bytes(), prefix, suffix);
}
}

View File

@ -0,0 +1,201 @@
package org.apache.lucene.codecs.blocktree;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.io.UnsupportedEncodingException;
import java.util.Locale;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
* BlockTree statistics for a single field
* returned by {@link Lucene40FieldReader#computeStats()}.
* @deprecated Only for 4.x backcompat
*/
@Deprecated
final class Lucene40Stats {
/** How many nodes in the index FST. */
public long indexNodeCount;
/** How many arcs in the index FST. */
public long indexArcCount;
/** Byte size of the index. */
public long indexNumBytes;
/** Total number of terms in the field. */
public long totalTermCount;
/** Total number of bytes (sum of term lengths) across all terms in the field. */
public long totalTermBytes;
/** The number of normal (non-floor) blocks in the terms file. */
public int nonFloorBlockCount;
/** The number of floor blocks (meta-blocks larger than the
* allowed {@code maxItemsPerBlock}) in the terms file. */
public int floorBlockCount;
/** The number of sub-blocks within the floor blocks. */
public int floorSubBlockCount;
/** The number of "internal" blocks (that have both
* terms and sub-blocks). */
public int mixedBlockCount;
/** The number of "leaf" blocks (blocks that have only
* terms). */
public int termsOnlyBlockCount;
/** The number of "internal" blocks that do not contain
* terms (have only sub-blocks). */
public int subBlocksOnlyBlockCount;
/** Total number of blocks. */
public int totalBlockCount;
/** Number of blocks at each prefix depth. */
public int[] blockCountByPrefixLen = new int[10];
private int startBlockCount;
private int endBlockCount;
/** Total number of bytes used to store term suffixes. */
public long totalBlockSuffixBytes;
/** Total number of bytes used to store term stats (not
* including what the {@link PostingsReaderBase}
* stores. */
public long totalBlockStatsBytes;
/** Total bytes stored by the {@link PostingsReaderBase},
* plus the other few vInts stored in the frame. */
public long totalBlockOtherBytes;
/** Segment name. */
public final String segment;
/** Field name. */
public final String field;
Lucene40Stats(String segment, String field) {
this.segment = segment;
this.field = field;
}
void startBlock(Lucene40SegmentTermsEnumFrame 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(Lucene40SegmentTermsEnumFrame frame) {
final int termCount = frame.isLeafBlock ? frame.entCount : frame.state.termBlockOrd;
final int subBlockCount = frame.entCount - termCount;
totalTermCount += termCount;
if (termCount != 0 && subBlockCount != 0) {
mixedBlockCount++;
} else if (termCount != 0) {
termsOnlyBlockCount++;
} else if (subBlockCount != 0) {
subBlocksOnlyBlockCount++;
} else {
throw new IllegalStateException();
}
endBlockCount++;
final long otherBytes = frame.fpEnd - frame.fp - frame.suffixesReader.length() - frame.statsReader.length();
assert otherBytes > 0 : "otherBytes=" + otherBytes + " frame.fp=" + frame.fp + " frame.fpEnd=" + frame.fpEnd;
totalBlockOtherBytes += otherBytes;
}
void term(BytesRef term) {
totalTermBytes += term.length;
}
void finish() {
assert startBlockCount == endBlockCount: "startBlockCount=" + startBlockCount + " endBlockCount=" + endBlockCount;
assert totalBlockCount == floorSubBlockCount + nonFloorBlockCount: "floorSubBlockCount=" + floorSubBlockCount + " nonFloorBlockCount=" + nonFloorBlockCount + " totalBlockCount=" + totalBlockCount;
assert totalBlockCount == mixedBlockCount + termsOnlyBlockCount + subBlocksOnlyBlockCount: "totalBlockCount=" + totalBlockCount + " mixedBlockCount=" + mixedBlockCount + " subBlocksOnlyBlockCount=" + subBlocksOnlyBlockCount + " termsOnlyBlockCount=" + termsOnlyBlockCount;
}
@Override
public String toString() {
final ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
PrintStream out;
try {
out = new PrintStream(bos, false, IOUtils.UTF_8);
} catch (UnsupportedEncodingException bogus) {
throw new RuntimeException(bogus);
}
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(Locale.ROOT, "%.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(Locale.ROOT, "%.1f", ((double) totalBlockSuffixBytes)/totalBlockCount) + " suffix-bytes/block)" : ""));
out.println(" " + totalBlockStatsBytes + " term stats bytes" + (totalBlockCount != 0 ? " (" + String.format(Locale.ROOT, "%.1f", ((double) totalBlockStatsBytes)/totalBlockCount) + " stats-bytes/block)" : ""));
out.println(" " + totalBlockOtherBytes + " other bytes" + (totalBlockCount != 0 ? " (" + String.format(Locale.ROOT, "%.1f", ((double) totalBlockOtherBytes)/totalBlockCount) + " other-bytes/block)" : ""));
if (totalBlockCount != 0) {
out.println(" by prefix length:");
int total = 0;
for(int prefix=0;prefix<blockCountByPrefixLen.length;prefix++) {
final int blockCount = blockCountByPrefixLen[prefix];
total += blockCount;
if (blockCount != 0) {
out.println(" " + String.format(Locale.ROOT, "%2d", prefix) + ": " + blockCount);
}
}
assert totalBlockCount == total;
}
try {
return bos.toString(IOUtils.UTF_8);
} catch (UnsupportedEncodingException bogus) {
throw new RuntimeException(bogus);
}
}
}

View File

@ -0,0 +1,26 @@
<!--
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.
-->
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
BlockTree terms dictionary from Lucene 4.0-4.10
</body>
</html>

View File

@ -23,7 +23,7 @@ import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.Lucene40BlockTreeTermsReader;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -51,7 +51,7 @@ public class Lucene40PostingsFormat extends PostingsFormat {
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(postings, state);
FieldsProducer ret = new Lucene40BlockTreeTermsReader(postings, state);
success = true;
return ret;
} finally {

View File

@ -23,7 +23,7 @@ import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.Lucene40BlockTreeTermsReader;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
@ -101,7 +101,7 @@ public class Lucene41PostingsFormat extends PostingsFormat {
state.segmentSuffix);
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(postingsReader, state);
FieldsProducer ret = new Lucene40BlockTreeTermsReader(postingsReader, state);
success = true;
return ret;
} finally {

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs.lucene41;
package org.apache.lucene.codecs.blocktree;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -19,8 +19,9 @@ package org.apache.lucene.codecs.lucene41;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.blocktree.FieldReader;
import org.apache.lucene.codecs.blocktree.Stats;
import org.apache.lucene.codecs.blocktree.Lucene40FieldReader;
import org.apache.lucene.codecs.blocktree.Lucene40Stats;
import org.apache.lucene.codecs.lucene41.Lucene41RWCodec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.BasePostingsFormatTestCase;
@ -32,7 +33,7 @@ import org.apache.lucene.store.Directory;
/**
* Tests BlockPostingsFormat
*/
public class TestLucene41PostingsFormat extends BasePostingsFormatTestCase {
public class TestLucene40BlockFormat extends BasePostingsFormatTestCase {
private final Codec codec = new Lucene41RWCodec();
@Override
@ -54,9 +55,9 @@ public class TestLucene41PostingsFormat extends BasePostingsFormatTestCase {
DirectoryReader r = DirectoryReader.open(w, true);
assertEquals(1, r.leaves().size());
FieldReader field = (FieldReader) r.leaves().get(0).reader().fields().terms("field");
Lucene40FieldReader field = (Lucene40FieldReader) r.leaves().get(0).reader().fields().terms("field");
// We should see exactly two blocks: one root block (prefix empty string) and one block for z* terms (prefix z):
Stats stats = field.computeStats();
Lucene40Stats stats = field.computeStats();
assertEquals(0, stats.floorBlockCount);
assertEquals(2, stats.nonFloorBlockCount);
r.close();

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.codecs.blocktree.Lucene40BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentWriteState;
/**
@ -46,7 +46,7 @@ public final class Lucene40RWPostingsFormat extends Lucene40PostingsFormat {
// Or... you must make a new Codec for this?
boolean success = false;
try {
FieldsConsumer ret = new BlockTreeTermsWriter(state, docs, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
FieldsConsumer ret = new Lucene40BlockTreeTermsWriter(state, docs, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
success = true;
return ret;
} finally {

View File

@ -21,7 +21,7 @@ import java.io.IOException;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.codecs.blocktree.Lucene40BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;
@ -41,7 +41,7 @@ public class Lucene41RWPostingsFormat extends Lucene41PostingsFormat {
boolean success = false;
try {
FieldsConsumer ret = new BlockTreeTermsWriter(state,
FieldsConsumer ret = new Lucene40BlockTreeTermsWriter(state,
postingsWriter,
MIN_BLOCK_SIZE,
MAX_BLOCK_SIZE);

View File

@ -115,9 +115,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
}
// verify
if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
CodecUtil.checksumEntireFile(indexIn);
}
CodecUtil.checksumEntireFile(indexIn);
// Have PostingsReader init itself
postingsReader.init(in, state);
@ -127,9 +125,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
// but for now we at least verify proper structure of the checksum footer: which looks
// for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
// such as file truncation.
if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
CodecUtil.retrieveChecksum(in);
}
CodecUtil.retrieveChecksum(in);
// Read per-field details
seekDir(in, dirOffset);
@ -160,17 +156,12 @@ public final class BlockTreeTermsReader extends FieldsProducer {
final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
final long sumDocFreq = in.readVLong();
final int docCount = in.readVInt();
final int longsSize = version >= BlockTreeTermsWriter.VERSION_META_ARRAY ? in.readVInt() : 0;
final int longsSize = in.readVInt();
if (longsSize < 0) {
throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, in);
}
BytesRef minTerm, maxTerm;
if (version >= BlockTreeTermsWriter.VERSION_MIN_MAX_TERMS) {
minTerm = readBytesRef(in);
maxTerm = readBytesRef(in);
} else {
minTerm = maxTerm = null;
}
BytesRef minTerm = readBytesRef(in);
BytesRef maxTerm = readBytesRef(in);
if (docCount < 0 || docCount > state.segmentInfo.getDocCount()) { // #docs with field must be <= #docs
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.getDocCount(), in);
}
@ -212,9 +203,6 @@ public final class BlockTreeTermsReader extends FieldsProducer {
int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_CODEC_NAME,
BlockTreeTermsWriter.VERSION_START,
BlockTreeTermsWriter.VERSION_CURRENT);
if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
dirOffset = input.readLong();
}
return version;
}
@ -223,22 +211,14 @@ public final class BlockTreeTermsReader extends FieldsProducer {
int version = CodecUtil.checkHeader(input, BlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
BlockTreeTermsWriter.VERSION_START,
BlockTreeTermsWriter.VERSION_CURRENT);
if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
indexDirOffset = input.readLong();
}
return version;
}
/** Seek {@code input} to the directory offset. */
private void seekDir(IndexInput input, long dirOffset)
throws IOException {
if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
input.seek(input.length() - CodecUtil.footerLength() - 8);
dirOffset = input.readLong();
} else if (version >= BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
input.seek(input.length() - 8);
dirOffset = input.readLong();
}
input.seek(input.length() - CodecUtil.footerLength() - 8);
dirOffset = input.readLong();
input.seek(dirOffset);
}
@ -308,14 +288,12 @@ public final class BlockTreeTermsReader extends FieldsProducer {
}
@Override
public void checkIntegrity() throws IOException {
if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
// term dictionary
CodecUtil.checksumEntireFile(in);
public void checkIntegrity() throws IOException {
// term dictionary
CodecUtil.checksumEntireFile(in);
// postings
postingsReader.checkIntegrity();
}
// postings
postingsReader.checkIntegrity();
}
@Override

View File

@ -220,21 +220,9 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
/** Initial terms format. */
public static final int VERSION_START = 0;
/** Append-only */
public static final int VERSION_APPEND_ONLY = 1;
/** Meta data as array */
public static final int VERSION_META_ARRAY = 2;
/** checksums */
public static final int VERSION_CHECKSUM = 3;
/** min/max term */
public static final int VERSION_MIN_MAX_TERMS = 4;
/** Current terms format. */
public static final int VERSION_CURRENT = VERSION_MIN_MAX_TERMS;
public static final int VERSION_CURRENT = VERSION_START;
/** Extension of terms index file */
static final String TERMS_INDEX_EXTENSION = "tip";

View File

@ -278,7 +278,7 @@ public class CheckIndex implements Closeable {
* tree terms dictionary (this is only set if the
* {@link PostingsFormat} for this segment uses block
* tree. */
public Map<String,Stats> blockTreeStats = null;
public Map<String,Object> blockTreeStats = null;
}
/**
@ -1289,14 +1289,12 @@ public class CheckIndex implements Closeable {
// docs got deleted and then merged away):
} else {
if (fieldTerms instanceof FieldReader) {
final Stats stats = ((FieldReader) fieldTerms).computeStats();
assert stats != null;
if (status.blockTreeStats == null) {
status.blockTreeStats = new HashMap<>();
}
status.blockTreeStats.put(field, stats);
final Object stats = fieldTerms.getStats();
assert stats != null;
if (status.blockTreeStats == null) {
status.blockTreeStats = new HashMap<>();
}
status.blockTreeStats.put(field, stats);
if (sumTotalTermFreq != 0) {
final long v = fields.terms(field).getSumTotalTermFreq();
@ -1423,7 +1421,7 @@ public class CheckIndex implements Closeable {
}
if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) {
for(Map.Entry<String,Stats> ent : status.blockTreeStats.entrySet()) {
for(Map.Entry<String, Object> ent : status.blockTreeStats.entrySet()) {
infoStream.println(" field \"" + ent.getKey() + "\":");
infoStream.println(" " + ent.getValue().toString().replace("\n", "\n "));
}

View File

@ -193,4 +193,9 @@ public abstract class Terms {
scratch.grow(scratch.length());
}
}
public String getStats() {
// nocommit: add a meaningful default
return "";
}
}