LUCENE-5675: add IDVersionPostingsFormat

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1596979 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2014-05-22 22:35:23 +00:00
commit 8a1587d6d8
46 changed files with 8420 additions and 3147 deletions

View File

@ -115,6 +115,10 @@ New Features
minimum numeric values from the provided Terms. (Robert Muir, Mike
McCandless)
* LUCENE-5675: Add IDVersionPostingsFormat, a postings format
optimized for primary-key (ID) fields that also record a version
(long) for each ID. (Robert Muir, Mike McCandless)
Changes in Backwards Compatibility Policy
* LUCENE-5634: Add reuse argument to IndexableField.tokenStream. This

View File

@ -17,7 +17,7 @@ package org.apache.lucene.codecs.pulsing;
* limitations under the License.
*/
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsBaseFormat;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs

View File

@ -19,14 +19,14 @@ package org.apache.lucene.codecs.pulsing;
import java.io.IOException;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsBaseFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.IOUtils;

View File

@ -37,6 +37,8 @@ import org.apache.lucene.util.Bits;
* time.
* @lucene.experimental */
// TODO: maybe move under blocktree? but it's used by other terms dicts (e.g. Block)
// TODO: find a better name; this defines the API that the
// terms dict impls use to talk to a postings impl.
// TermsDict + PostingsReader/WriterBase == PostingsConsumer/Producer

View File

@ -20,6 +20,7 @@ package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.DocsAndPositionsEnum; // javadocs
import org.apache.lucene.index.DocsEnum; // javadocs
import org.apache.lucene.index.FieldInfo;

View File

@ -0,0 +1,313 @@
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.io.PrintStream;
import java.util.Collections;
import java.util.Iterator;
import java.util.TreeMap;
import org.apache.lucene.codecs.BlockTermState;
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.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.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;
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.IOUtils;
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.automaton.Transition;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.Outputs;
import org.apache.lucene.util.fst.Util;
/** A block-based terms index and dictionary that assigns
* terms to variable length blocks according to how they
* share prefixes. The terms index is a prefix trie
* whose leaves are term blocks. The advantage of this
* approach is that 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 BlockTreeTermsWriter}.
*
* @lucene.experimental
*/
public final class BlockTreeTermsReader 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,FieldReader> 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 BlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
PostingsReaderBase postingsReader, IOContext ioContext,
String segmentSuffix)
throws IOException {
this.postingsReader = postingsReader;
this.segment = info.name;
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_EXTENSION),
ioContext);
boolean success = false;
IndexInput indexIn = null;
try {
version = readHeader(in);
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, BlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
ioContext);
int indexVersion = readIndexHeader(indexIn);
if (indexVersion != version) {
throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
}
// verify
if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
CodecUtil.checksumEntireFile(indexIn);
}
// Have PostingsReader init itself
postingsReader.init(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 + " (resource=" + in + ")");
}
for(int i=0;i<numFields;i++) {
final int field = in.readVInt();
final long numTerms = in.readVLong();
assert numTerms >= 0;
final int numBytes = in.readVInt();
final BytesRef rootCode = new BytesRef(new byte[numBytes]);
in.readBytes(rootCode.bytes, 0, numBytes);
rootCode.length = numBytes;
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
assert fieldInfo != null: "field=" + field;
final long sumTotalTermFreq = fieldInfo.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;
BytesRef minTerm, maxTerm;
if (version >= BlockTreeTermsWriter.VERSION_MIN_MAX_TERMS) {
minTerm = readBytesRef(in);
maxTerm = readBytesRef(in);
} else {
minTerm = maxTerm = null;
}
if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount() + " (resource=" + in + ")");
}
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount + " (resource=" + in + ")");
}
if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")");
}
final long indexStartFP = indexIn.readVLong();
FieldReader previous = fields.put(fieldInfo.name,
new FieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount,
indexStartFP, longsSize, indexIn, minTerm, maxTerm));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + 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, BlockTreeTermsWriter.TERMS_CODEC_NAME,
BlockTreeTermsWriter.VERSION_START,
BlockTreeTermsWriter.VERSION_CURRENT);
if (version < BlockTreeTermsWriter.VERSION_APPEND_ONLY) {
dirOffset = input.readLong();
}
return version;
}
/** Reads index file header. */
private int readIndexHeader(IndexInput input) throws IOException {
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(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 sizeInByes = ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0);
for(FieldReader reader : fields.values()) {
sizeInByes += reader.ramBytesUsed();
}
return sizeInByes;
}
@Override
public void checkIntegrity() throws IOException {
if (version >= BlockTreeTermsWriter.VERSION_CHECKSUM) {
// term dictionary
CodecUtil.checksumEntireFile(in);
// postings
postingsReader.checkIntegrity();
}
}
}

View File

@ -1,4 +1,4 @@
package org.apache.lucene.codecs;
package org.apache.lucene.codecs.blocktree;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -21,6 +21,10 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.codecs.BlockTermState;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
@ -42,6 +46,7 @@ import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.NoOutputs;
import org.apache.lucene.util.fst.Outputs;
import org.apache.lucene.util.fst.Util;
import org.apache.lucene.util.packed.PackedInts;
@ -183,7 +188,11 @@ import org.apache.lucene.util.packed.PackedInts;
* @see BlockTreeTermsReader
* @lucene.experimental
*/
public class BlockTreeTermsWriter extends FieldsConsumer {
public final class BlockTreeTermsWriter extends FieldsConsumer {
static final Outputs<BytesRef> FST_OUTPUTS = ByteSequenceOutputs.getSingleton();
static final BytesRef NO_OUTPUT = FST_OUTPUTS.getNoOutput();
/** Suggested default value for the {@code
* minItemsInBlock} parameter to {@link
@ -597,14 +606,14 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
}
// Write the top count entries on the pending stack as
// one or more blocks. Returns how many blocks were
// written. If the entry count is <= maxItemsPerBlock
// one or more blocks. If the entry count is <= maxItemsPerBlock
// we just write a single block; else we break into
// primary (initial) block and then one or more
// following floor blocks:
void writeBlocks(IntsRef prevTerm, int prefixLength, int count) throws IOException {
if (prefixLength == 0 || count <= maxItemsInBlock) {
// System.out.println("writeBlocks count=" + count);
if (count <= maxItemsInBlock) {
// Easy case: not floor block. Eg, prefix is "foo",
// and we found 30 terms/sub-blocks starting w/ that
// prefix, and minItemsInBlock <= 30 <=
@ -612,6 +621,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
final PendingBlock nonFloorBlock = writeBlock(prevTerm, prefixLength, prefixLength, count, count, 0, false, -1, true);
nonFloorBlock.compileIndex(null, scratchBytes);
pending.add(nonFloorBlock);
// System.out.println(" 1 block");
} else {
// Floor block case. Eg, prefix is "foo" but we
// have 100 terms/sub-blocks starting w/ that
@ -768,6 +778,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
floorBlocks.add(floorBlock);
}
curStart -= pendingCount;
// System.out.println(" floor=" + pendingCount);
//System.out.println(" = " + pendingCount);
pendingCount = 0;
@ -1041,12 +1052,12 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
// terms into "good" blocks; we don't save the
// resulting FST:
blockBuilder = new Builder<>(FST.INPUT_TYPE.BYTE1,
0, 0, true,
true, Integer.MAX_VALUE,
noOutputs,
new FindBlocks(), false,
PackedInts.COMPACT,
true, 15);
0, 0, true,
true, Integer.MAX_VALUE,
noOutputs,
new FindBlocks(), false,
PackedInts.COMPACT,
true, 15);
this.longsSize = postingsWriter.setField(fieldInfo);
}

View File

@ -0,0 +1,176 @@
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.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
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.BytesRef;
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}. */
// public for CheckIndex:
public final class FieldReader extends Terms {
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 BlockTreeTermsReader parent;
final FST<BytesRef> index;
//private boolean DEBUG;
FieldReader(BlockTreeTermsReader 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() >>> BlockTreeTermsWriter.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 Stats computeStats() throws IOException {
return new SegmentTermsEnum(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 SegmentTermsEnum(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 IntersectTermsEnum(this, compiled, startTerm);
}
/** Returns approximate RAM bytes used */
public long ramBytesUsed() {
return ((index!=null)? index.sizeInBytes() : 0);
}
}

View File

@ -0,0 +1,484 @@
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!
final class IntersectTermsEnum extends TermsEnum {
final IndexInput in;
final static Outputs<BytesRef> fstOutputs = ByteSequenceOutputs.getSingleton();
private IntersectTermsEnumFrame[] stack;
@SuppressWarnings({"rawtypes","unchecked"}) private FST.Arc<BytesRef>[] arcs = new FST.Arc[5];
final RunAutomaton runAutomaton;
final CompiledAutomaton compiledAutomaton;
private IntersectTermsEnumFrame currentFrame;
private final BytesRef term = new BytesRef();
private final FST.BytesReader fstReader;
final FieldReader 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 IntersectTermsEnum(FieldReader 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 IntersectTermsEnumFrame[5];
for(int idx=0;idx<stack.length;idx++) {
stack[idx] = new IntersectTermsEnumFrame(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 IntersectTermsEnumFrame 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 IntersectTermsEnumFrame getFrame(int ord) throws IOException {
if (ord >= stack.length) {
final IntersectTermsEnumFrame[] next = new IntersectTermsEnumFrame[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 IntersectTermsEnumFrame(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 IntersectTermsEnumFrame pushFrame(int state) throws IOException {
final IntersectTermsEnumFrame 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.transitions.length-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++;
currentFrame.curTransitionMax = currentFrame.transitions[currentFrame.transitionIndex].getMax();
//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,292 @@
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;
// TODO: can we share this with the frame in STE?
final class IntersectTermsEnumFrame {
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<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 IntersectTermsEnum ite;
public IntersectTermsEnumFrame(IntersectTermsEnum 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 <= transitions[transitionIndex].getMin());
load(null);
}
public void setState(int state) {
this.state = state;
transitionIndex = 0;
transitions = ite.compiledAutomaton.sortedTransitions[state];
if (transitions.length != 0) {
curTransitionMax = transitions[0].getMax();
} else {
curTransitionMax = -1;
}
}
void load(BytesRef frameIndexData) throws IOException {
// if (DEBUG) System.out.println(" load fp=" + fp + " fpOrig=" + fpOrig + " frameIndexData=" + frameIndexData + " trans=" + (transitions.length != 0 ? transitions[0] : "n/a" + " state=" + state));
if (frameIndexData != null && transitions.length != 0) {
// Floor frame
if (floorData.length < frameIndexData.length) {
this.floorData = new byte[ArrayUtil.oversize(frameIndexData.length, 1)];
}
System.arraycopy(frameIndexData.bytes, frameIndexData.offset, floorData, 0, frameIndexData.length);
floorDataReader.reset(floorData, 0, frameIndexData.length);
// Skip first long -- has redundant fp, hasTerms
// flag, isFloor flag
final long code = floorDataReader.readVLong();
if ((code & BlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) {
numFollowFloorBlocks = floorDataReader.readVInt();
nextFloorLabel = floorDataReader.readByte() & 0xff;
// if (DEBUG) System.out.println(" numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
// If current state is accept, we must process
// first block in case it has empty suffix:
if (!ite.runAutomaton.isAccept(state)) {
// Maybe skip floor blocks:
while (numFollowFloorBlocks != 0 && nextFloorLabel <= transitions[0].getMin()) {
fp = fpOrig + (floorDataReader.readVLong() >>> 1);
numFollowFloorBlocks--;
// if (DEBUG) System.out.println(" skip floor block! nextFloorLabel=" + (char) nextFloorLabel + " vs target=" + (char) transitions[0].getMin() + " newFP=" + fp + " numFollowFloorBlocks=" + numFollowFloorBlocks);
if (numFollowFloorBlocks != 0) {
nextFloorLabel = floorDataReader.readByte() & 0xff;
} else {
nextFloorLabel = 256;
}
}
}
}
}
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;
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,745 @@
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;
final class SegmentTermsEnumFrame {
// 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 SegmentTermsEnum ste;
public SegmentTermsEnumFrame(SegmentTermsEnum 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);
// 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();
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.length = prefix + suffix;
if (ste.term.bytes.length < ste.term.length) {
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.length = prefix + suffix;
if (ste.term.bytes.length < ste.term.length) {
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.bytes[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 (!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;
}
// 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.length = prefix + suffix;
if (ste.term.bytes.length < termLength) {
ste.term.grow(termLength);
}
System.arraycopy(suffixBytes, startBytePos, ste.term.bytes, prefix, suffix);
}
}

View File

@ -0,0 +1,199 @@
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.PostingsBaseFormat;
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 FieldReader#computeStats()}.
*/
public class Stats {
/** 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 PostingsBaseFormat}
* stores. */
public long totalBlockStatsBytes;
/** Total bytes stored by the {@link PostingsBaseFormat},
* plus the other few vInts stored in the frame. */
public long totalBlockOtherBytes;
/** Segment name. */
public final String segment;
/** Field name. */
public final String field;
Stats(String segment, String field) {
this.segment = segment;
this.field = field;
}
void startBlock(SegmentTermsEnumFrame 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(SegmentTermsEnumFrame 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,38 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
BlockTree terms dictionary.
<p>
This terms dictionary organizes all terms into blocks according to
shared prefix, such that each block has enough terms, and then stores
the prefix trie in memory as an FST as the index structure. It allows
you to plug in your own {@link
org.apache.lucene.codecs.PostingsBaseFormat} to implement the
postings.
</p>
<p>See {@link org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter}
for the file format.
</p>
</body>
</html>

View File

@ -19,14 +19,14 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.CodecUtil;
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.PostingsWriterBase; // javadocs
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.DocsEnum; // javadocs
import org.apache.lucene.index.FieldInfo.IndexOptions; // javadocs
import org.apache.lucene.index.FieldInfos; // javadocs

View File

@ -20,8 +20,6 @@ package org.apache.lucene.codecs.lucene41;
import java.io.IOException;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
@ -29,6 +27,8 @@ import org.apache.lucene.codecs.MultiLevelSkipListWriter;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.SegmentReadState;

View File

@ -127,6 +127,8 @@ class BufferedUpdates {
final AtomicInteger numTermDeletes = new AtomicInteger();
final AtomicInteger numNumericUpdates = new AtomicInteger();
final AtomicInteger numBinaryUpdates = new AtomicInteger();
// TODO: rename thes three: put "deleted" prefix in front:
final Map<Term,Integer> terms = new HashMap<>();
final Map<Query,Integer> queries = new HashMap<>();
final List<Integer> docIDs = new ArrayList<>();

View File

@ -28,9 +28,11 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.FieldReader;
import org.apache.lucene.codecs.blocktree.Stats;
import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.search.DocIdSetIterator;
@ -45,6 +47,7 @@ import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LongBitSet;
import org.apache.lucene.util.StringHelper;
/**
* Basic tool and API to check the health of an index and
* write a new segments file that removes reference to
@ -237,7 +240,7 @@ public class CheckIndex {
* tree terms dictionary (this is only set if the
* {@link PostingsFormat} for this segment uses block
* tree. */
public Map<String,BlockTreeTermsReader.Stats> blockTreeStats = null;
public Map<String,Stats> blockTreeStats = null;
}
/**
@ -1115,8 +1118,8 @@ public class CheckIndex {
// docs got deleted and then merged away):
} else {
if (fieldTerms instanceof BlockTreeTermsReader.FieldReader) {
final BlockTreeTermsReader.Stats stats = ((BlockTreeTermsReader.FieldReader) fieldTerms).computeStats();
if (fieldTerms instanceof FieldReader) {
final Stats stats = ((FieldReader) fieldTerms).computeStats();
assert stats != null;
if (status.blockTreeStats == null) {
status.blockTreeStats = new HashMap<>();
@ -1249,7 +1252,7 @@ public class CheckIndex {
}
if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) {
for(Map.Entry<String,BlockTreeTermsReader.Stats> ent : status.blockTreeStats.entrySet()) {
for(Map.Entry<String,Stats> ent : status.blockTreeStats.entrySet()) {
infoStream.println(" field \"" + ent.getKey() + "\":");
infoStream.println(" " + ent.getValue().toString().replace("\n", "\n "));
}
@ -1639,6 +1642,7 @@ public class CheckIndex {
// Only agg stats if the doc is live:
final boolean doStats = liveDocs == null || liveDocs.get(j);
if (doStats) {
status.docCount++;
}

View File

@ -34,6 +34,7 @@ final class FreqProxTermsWriter extends TermsHash {
}
private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException {
// Process any pending Term deletes for this newly
// flushed segment:
if (state.segUpdates != null && state.segUpdates.terms.size() > 0) {
@ -57,11 +58,9 @@ final class FreqProxTermsWriter extends TermsHash {
if (termsEnum != null && termsEnum.seekExact(deleteTerm.bytes())) {
docsEnum = termsEnum.docs(null, docsEnum, 0);
int delDocLimit = segDeletes.get(deleteTerm);
assert delDocLimit < DocsEnum.NO_MORE_DOCS;
while (true) {
int doc = docsEnum.nextDoc();
if (doc == DocsEnum.NO_MORE_DOCS) {
break;
}
if (doc < delDocLimit) {
if (state.liveDocs == null) {
state.liveDocs = state.segmentInfo.getCodec().liveDocsFormat().newLiveDocs(state.segmentInfo.getDocCount());

View File

@ -105,5 +105,6 @@ public class SegmentWriteState {
this.segmentSuffix = segmentSuffix;
segUpdates = state.segUpdates;
delCountOnFlush = state.delCountOnFlush;
liveDocs = state.liveDocs;
}
}

View File

@ -61,6 +61,11 @@ public class PairOutputs<A,B> extends Outputs<PairOutputs.Pair<A,B>> {
public int hashCode() {
return output1.hashCode() + output2.hashCode();
}
@Override
public String toString() {
return "Pair(" + output1 + "," + output2 + ")";
}
};
public PairOutputs(Outputs<A> outputs1, Outputs<B> outputs2) {

View File

@ -967,73 +967,41 @@ public class TestTermsEnum extends LuceneTestCase {
d.close();
}
/** Utility class to do efficient primary-key (only 1 doc contains the
* given term) lookups by segment, re-using the enums. This class is
* not thread safe, so it is the caller's job to create and use one
* instance of this per thread. Do not use this if a term may appear
* in more than one document! It will only return the first one it
* finds. */
static class PerThreadPKLookup {
private final TermsEnum[] termsEnums;
private final DocsEnum[] docsEnums;
private final Bits[] liveDocs;
private final int[] docBases;
private final int numSegs;
private final boolean hasDeletions;
public PerThreadPKLookup(IndexReader r, String idFieldName) throws IOException {
List<AtomicReaderContext> leaves = new ArrayList<>(r.leaves());
// Larger segments are more likely to have the id, so we sort largest to smallest by numDocs:
Collections.sort(leaves, new Comparator<AtomicReaderContext>() {
@Override
public int compare(AtomicReaderContext c1, AtomicReaderContext c2) {
return c2.reader().numDocs() - c1.reader().numDocs();
}
});
termsEnums = new TermsEnum[leaves.size()];
docsEnums = new DocsEnum[leaves.size()];
liveDocs = new Bits[leaves.size()];
docBases = new int[leaves.size()];
int numSegs = 0;
boolean hasDeletions = false;
for(int i=0;i<leaves.size();i++) {
Fields fields = leaves.get(i).reader().fields();
if (fields != null) {
Terms terms = fields.terms(idFieldName);
if (terms != null) {
termsEnums[numSegs] = terms.iterator(null);
assert termsEnums[numSegs] != null;
docBases[numSegs] = leaves.get(i).docBase;
liveDocs[numSegs] = leaves.get(i).reader().getLiveDocs();
hasDeletions |= leaves.get(i).reader().hasDeletions();
numSegs++;
}
}
}
this.numSegs = numSegs;
this.hasDeletions = hasDeletions;
// Stresses out many-terms-in-root-block case:
@Slow
public void testVaryingTermsPerSegment() throws Exception {
Directory dir = newDirectory();
Set<BytesRef> terms = new HashSet<BytesRef>();
int MAX_TERMS = atLeast(1000);
while (terms.size() < MAX_TERMS) {
terms.add(new BytesRef(TestUtil.randomSimpleString(random(), 1, 40)));
}
/** Returns docID if found, else -1. */
public int lookup(BytesRef id) throws IOException {
for(int seg=0;seg<numSegs;seg++) {
if (termsEnums[seg].seekExact(id)) {
docsEnums[seg] = termsEnums[seg].docs(liveDocs[seg], docsEnums[seg], 0);
int docID = docsEnums[seg].nextDoc();
if (docID != DocsEnum.NO_MORE_DOCS) {
return docBases[seg] + docID;
}
assert hasDeletions;
}
List<BytesRef> termsList = new ArrayList<>(terms);
StringBuilder sb = new StringBuilder();
for(int termCount=0;termCount<MAX_TERMS;termCount++) {
if (VERBOSE) {
System.out.println("\nTEST: termCount=" + termCount + " add term=" + termsList.get(termCount).utf8ToString());
}
return -1;
sb.append(' ');
sb.append(termsList.get(termCount).utf8ToString());
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(newTextField("field", sb.toString(), Field.Store.NO));
w.addDocument(doc);
IndexReader r = w.getReader();
assertEquals(1, r.leaves().size());
TermsEnum te = r.leaves().get(0).reader().fields().terms("field").iterator(null);
for(int i=0;i<=termCount;i++) {
assertTrue("term '" + termsList.get(i).utf8ToString() + "' should exist but doesn't", te.seekExact(termsList.get(i)));
}
for(int i=termCount+1;i<termsList.size();i++) {
assertFalse("term '" + termsList.get(i) + "' shouldn't exist but does", te.seekExact(termsList.get(i)));
}
r.close();
w.shutdown();
}
// TODO: add reopen method to carry over re-used enums...?
dir.close();
}
}

View File

@ -623,8 +623,14 @@ public class TestBlockJoin extends LuceneTestCase {
System.out.println("TEST: reader=" + r);
System.out.println("TEST: joinReader=" + joinR);
Bits liveDocs = MultiFields.getLiveDocs(joinR);
for(int docIDX=0;docIDX<joinR.maxDoc();docIDX++) {
System.out.println(" docID=" + docIDX + " doc=" + joinR.document(docIDX));
System.out.println(" docID=" + docIDX + " doc=" + joinR.document(docIDX) + " deleted?=" + (liveDocs != null && liveDocs.get(docIDX) == false));
}
DocsEnum parents = MultiFields.getTermDocsEnum(joinR, null, "isParent", new BytesRef("x"));
System.out.println("parent docIDs:");
while (parents.nextDoc() != parents.NO_MORE_DOCS) {
System.out.println(" " + parents.docID());
}
}
@ -823,6 +829,7 @@ public class TestBlockJoin extends LuceneTestCase {
Explanation explanation = joinS.explain(childJoinQuery, hit.doc);
StoredDocument document = joinS.doc(hit.doc - 1);
int childId = Integer.parseInt(document.get("childID"));
//System.out.println(" hit docID=" + hit.doc + " childId=" + childId + " parentId=" + document.get("parentID"));
assertTrue(explanation.isMatch());
assertEquals(hit.score, explanation.getValue(), 0.0f);
assertEquals(String.format(Locale.ROOT, "Score based on child doc range from %d to %d", hit.doc - 1 - childId, hit.doc - 1), explanation.getDescription());

View File

@ -514,8 +514,8 @@ public class TestFieldCacheVsDocValues extends LuceneTestCase {
BytesRef expectedBytes = new BytesRef();
BytesRef actualBytes = new BytesRef();
for (long i = 0; i < expected.getValueCount(); i++) {
expected.lookupTerm(expectedBytes);
actual.lookupTerm(actualBytes);
expected.lookupOrd(i, expectedBytes);
actual.lookupOrd(i, actualBytes);
assertEquals(expectedBytes, actualBytes);
}

View File

@ -0,0 +1,147 @@
package org.apache.lucene.codecs.idversion;
/*
* 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.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.search.LiveFieldValues;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/** A PostingsFormat optimized for primary-key (ID) fields that also
* record a version (long) for each ID, delivered as a payload
* created by {@link #longToBytes} during indexing. At search time,
* the TermsEnum implementation {@link IDVersionSegmentTermsEnum}
* enables fast (using only the terms index when possible) lookup for
* whether a given ID was previously indexed with version > N (see
* {@link IDVersionSegmentTermsEnum#seekExact(BytesRef,long)}.
*
* <p>This is most effective if the app assigns monotonically
* increasing global version to each indexed doc. Then, during
* indexing, use {@link
* IDVersionSegmentTermsEnum#seekExact(BytesRef,long)} (along with
* {@link LiveFieldValues}) to decide whether the document you are
* about to index was already indexed with a higher version, and skip
* it if so.
*
* <p>The field is effectively indexed as DOCS_ONLY and the docID is
* pulsed into the terms dictionary, but the user must feed in the
* version as a payload on the first token.
*
* <p>NOTE: term vectors cannot be indexed with this field (not that
* you should really ever want to do this).
*
* @lucene.experimental */
public class IDVersionPostingsFormat extends PostingsFormat {
/** version must be >= this. */
public static final long MIN_VERSION = 0;
// TODO: we could delta encode instead, and keep the last bit:
/** version must be <= this, because we encode with ZigZag. */
public static final long MAX_VERSION = 0x3fffffffffffffffL;
private final int minTermsInBlock;
private final int maxTermsInBlock;
public IDVersionPostingsFormat() {
this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
}
public IDVersionPostingsFormat(int minTermsInBlock, int maxTermsInBlock) {
super("IDVersion");
this.minTermsInBlock = minTermsInBlock;
this.maxTermsInBlock = maxTermsInBlock;
}
@Override
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
PostingsWriterBase postingsWriter = new IDVersionPostingsWriter(state);
boolean success = false;
try {
FieldsConsumer ret = new VersionBlockTreeTermsWriter(state,
postingsWriter,
minTermsInBlock,
maxTermsInBlock);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsWriter);
}
}
}
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new IDVersionPostingsReader();
boolean success = false;
try {
FieldsProducer ret = new VersionBlockTreeTermsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
postingsReader,
state.context,
state.segmentSuffix);
success = true;
return ret;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(postingsReader);
}
}
}
public static long bytesToLong(BytesRef bytes) {
return ((bytes.bytes[bytes.offset]&0xFFL) << 56) |
((bytes.bytes[bytes.offset+1]&0xFFL) << 48) |
((bytes.bytes[bytes.offset+2]&0xFFL) << 40) |
((bytes.bytes[bytes.offset+3]&0xFFL) << 32) |
((bytes.bytes[bytes.offset+4]&0xFFL) << 24) |
((bytes.bytes[bytes.offset+5]&0xFFL) << 16) |
((bytes.bytes[bytes.offset+6]&0xFFL) << 8) |
(bytes.bytes[bytes.offset+7]&0xFFL);
}
public static void longToBytes(long v, BytesRef bytes) {
if (v > MAX_VERSION || v < MIN_VERSION) {
throw new IllegalArgumentException("version must be >= MIN_VERSION=" + MIN_VERSION + " and <= MAX_VERSION=" + MAX_VERSION + " (got: " + v + ")");
}
bytes.offset = 0;
bytes.length = 8;
bytes.bytes[0] = (byte) (v >> 56);
bytes.bytes[1] = (byte) (v >> 48);
bytes.bytes[2] = (byte) (v >> 40);
bytes.bytes[3] = (byte) (v >> 32);
bytes.bytes[4] = (byte) (v >> 24);
bytes.bytes[5] = (byte) (v >> 16);
bytes.bytes[6] = (byte) (v >> 8);
bytes.bytes[7] = (byte) v;
assert bytesToLong(bytes) == v: bytesToLong(bytes) + " vs " + v + " bytes=" + bytes;
}
}

View File

@ -0,0 +1,102 @@
package org.apache.lucene.codecs.idversion;
/*
* 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.codecs.CodecUtil;
import org.apache.lucene.codecs.PostingsReaderBase;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.Bits;
final class IDVersionPostingsReader extends PostingsReaderBase {
@Override
public void init(IndexInput termsIn) throws IOException {
// Make sure we are talking to the matching postings writer
CodecUtil.checkHeader(termsIn,
IDVersionPostingsWriter.TERMS_CODEC,
IDVersionPostingsWriter.VERSION_START,
IDVersionPostingsWriter.VERSION_CURRENT);
}
@Override
public BlockTermState newTermState() {
return new IDVersionTermState();
}
@Override
public void close() throws IOException {
}
@Override
public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
throws IOException {
final IDVersionTermState termState = (IDVersionTermState) _termState;
termState.docID = in.readVInt();
if (absolute) {
termState.idVersion = in.readVLong();
} else {
termState.idVersion += BitUtil.zigZagDecode(in.readVLong());
}
}
@Override
public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
SingleDocsEnum docsEnum;
if (reuse instanceof SingleDocsEnum) {
docsEnum = (SingleDocsEnum) reuse;
} else {
docsEnum = new SingleDocsEnum();
}
docsEnum.reset(((IDVersionTermState) termState).docID, liveDocs);
return docsEnum;
}
@Override
public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs,
DocsAndPositionsEnum reuse, int flags) {
SingleDocsAndPositionsEnum posEnum;
if (reuse instanceof SingleDocsAndPositionsEnum) {
posEnum = (SingleDocsAndPositionsEnum) reuse;
} else {
posEnum = new SingleDocsAndPositionsEnum();
}
IDVersionTermState termState = (IDVersionTermState) _termState;
posEnum.reset(termState.docID, termState.idVersion, liveDocs);
return posEnum;
}
@Override
public long ramBytesUsed() {
return 0;
}
@Override
public void checkIntegrity() throws IOException {
}
}

View File

@ -0,0 +1,169 @@
package org.apache.lucene.codecs.idversion;
/*
* 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.codecs.CodecUtil;
import org.apache.lucene.codecs.PushPostingsWriterBase;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.BytesRef;
final class IDVersionPostingsWriter extends PushPostingsWriterBase {
final static String TERMS_CODEC = "IDVersionPostingsWriterTerms";
// Increment version to change it
final static int VERSION_START = 0;
final static int VERSION_CURRENT = VERSION_START;
final static IDVersionTermState emptyState = new IDVersionTermState();
IDVersionTermState lastState;
int lastDocID;
private int lastPosition;
private long lastVersion;
private final SegmentWriteState state;
public IDVersionPostingsWriter(SegmentWriteState state) {
this.state = state;
}
@Override
public BlockTermState newTermState() {
return new IDVersionTermState();
}
@Override
public void init(IndexOutput termsOut) throws IOException {
CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT);
}
@Override
public int setField(FieldInfo fieldInfo) {
super.setField(fieldInfo);
if (fieldInfo.getIndexOptions() != FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
throw new IllegalArgumentException("field must be index using IndexOptions.DOCS_AND_FREQS_AND_POSITIONS");
}
// LUCENE-5693: because CheckIndex cross-checks term vectors with postings even for deleted docs, and because our PF only indexes the
// non-deleted documents on flush, CheckIndex will see this as corruption:
if (fieldInfo.hasVectors()) {
throw new IllegalArgumentException("field cannot index term vectors: CheckIndex will report this as index corruption");
}
lastState = emptyState;
return 0;
}
@Override
public void startTerm() {
lastDocID = -1;
}
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
// TODO: LUCENE-5693: we don't need this check if we fix IW to not send deleted docs to us on flush:
if (state.liveDocs != null && state.liveDocs.get(docID) == false) {
return;
}
if (lastDocID != -1) {
throw new IllegalArgumentException("term appears in more than one document");
}
if (termDocFreq != 1) {
throw new IllegalArgumentException("term appears more than once in the document");
}
lastDocID = docID;
lastPosition = -1;
lastVersion = -1;
}
@Override
public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
if (lastDocID == -1) {
// Doc is deleted; skip it
return;
}
if (lastPosition != -1) {
throw new IllegalArgumentException("term appears more than once in document");
}
lastPosition = position;
if (payload == null) {
throw new IllegalArgumentException("token doens't have a payload");
}
if (payload.length != 8) {
throw new IllegalArgumentException("payload.length != 8 (got " + payload.length + ")");
}
lastVersion = IDVersionPostingsFormat.bytesToLong(payload);
if (lastVersion < IDVersionPostingsFormat.MIN_VERSION) {
throw new IllegalArgumentException("version must be >= MIN_VERSION=" + IDVersionPostingsFormat.MIN_VERSION + " (got: " + lastVersion + "; payload=" + payload + ")");
}
if (lastVersion > IDVersionPostingsFormat.MAX_VERSION) {
throw new IllegalArgumentException("version must be <= MAX_VERSION=" + IDVersionPostingsFormat.MAX_VERSION + " (got: " + lastVersion + "; payload=" + payload + ")");
}
}
@Override
public void finishDoc() throws IOException {
if (lastDocID == -1) {
// Doc is deleted; skip it
return;
}
if (lastPosition == -1) {
throw new IllegalArgumentException("missing addPosition");
}
}
/** Called when we are done adding docs to this term */
@Override
public void finishTerm(BlockTermState _state) throws IOException {
if (lastDocID == -1) {
return;
}
IDVersionTermState state = (IDVersionTermState) _state;
assert state.docFreq > 0;
state.docID = lastDocID;
state.idVersion = lastVersion;
}
private long lastEncodedVersion;
@Override
public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
IDVersionTermState state = (IDVersionTermState) _state;
out.writeVInt(state.docID);
if (absolute) {
out.writeVLong(state.idVersion);
} else {
long delta = state.idVersion - lastEncodedVersion;
out.writeVLong(BitUtil.zigZagEncode(delta));
}
lastEncodedVersion = state.idVersion;
}
@Override
public void close() throws IOException {
}
}

View File

@ -0,0 +1,738 @@
package org.apache.lucene.codecs.idversion;
/*
* 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;
import org.apache.lucene.util.fst.PairOutputs.Pair;
final class IDVersionSegmentTermsEnumFrame {
// Our index in stack[]:
final int ord;
boolean hasTerms;
boolean hasTermsOrig;
boolean isFloor;
// static boolean DEBUG = IDVersionSegmentTermsEnum.DEBUG;
/** Highest version of any term in this block. */
long maxIDVersion;
FST.Arc<Pair<BytesRef,Long>> 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[] 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 IDVersionSegmentTermsEnum ste;
public IDVersionSegmentTermsEnumFrame(IDVersionSegmentTermsEnum 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);
// 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);
}
}*/
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();
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.length = prefix + suffix;
if (ste.term.bytes.length < ste.term.length) {
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.length = prefix + suffix;
if (ste.term.bytes.length < ste.term.length) {
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=" + ((char) targetLabel) + " vs nextFloorLabel=" + ((char) 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=" + ((char) nextFloorLabel) + " fp=" + newFP + " hasTerms?=" + hasTerms + " numFollowFloor=" + numFollowFloorBlocks);
// }
isLastInFloor = numFollowFloorBlocks == 1;
numFollowFloorBlocks--;
if (isLastInFloor) {
nextFloorLabel = 256;
// if (DEBUG) {
// System.out.println(" stop! last block nextFloorLabel=" + ((char) nextFloorLabel));
// }
break;
} else {
nextFloorLabel = floorDataReader.readByte() & 0xff;
if (targetLabel < nextFloorLabel) {
// if (DEBUG) {
// System.out.println(" stop! nextFloorLabel=" + ((char) 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=" + ste.fr.parent.segment + " mdUpto=" + metaDataUpto + " vs termBlockOrd=" + state.termBlockOrd);
assert nextEnt >= 0;
// lazily catch up on metadata decode:
final int limit = getTermBlockOrd();
boolean absolute = metaDataUpto == 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 = 1;
state.totalTermFreq = 1;
//if (DEBUG) System.out.println(" dF=" + state.docFreq);
// 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.bytes[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=" + IDVersionSegmentTermsEnum.brToString(target) + " term=" + IDVersionSegmentTermsEnum.brToString(ste.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=" + IDVersionSegmentTermsEnum.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 (!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;
}
// 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=" + IDVersionSegmentTermsEnum.brToString(target) + " term=" + IDVersionSegmentTermsEnum.brToString(ste.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.length = prefix + suffix;
if (ste.term.bytes.length < termLength) {
ste.term.grow(termLength);
}
System.arraycopy(suffixBytes, startBytePos, ste.term.bytes, prefix, suffix);
}
}

View File

@ -0,0 +1,41 @@
package org.apache.lucene.codecs.idversion;
/*
* 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.codecs.BlockTermState;
import org.apache.lucene.index.TermState;
final class IDVersionTermState extends BlockTermState {
long idVersion;
int docID;
@Override
public IDVersionTermState clone() {
IDVersionTermState other = new IDVersionTermState();
other.copyFrom(this);
return other;
}
@Override
public void copyFrom(TermState _other) {
super.copyFrom(_other);
IDVersionTermState other = (IDVersionTermState) _other;
idVersion = other.idVersion;
docID = other.docID;
}
}

View File

@ -0,0 +1,105 @@
package org.apache.lucene.codecs.idversion;
/*
* 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.DocsAndPositionsEnum;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
class SingleDocsAndPositionsEnum extends DocsAndPositionsEnum {
private int doc;
private int pos;
private int singleDocID;
private Bits liveDocs;
private long version;
private final BytesRef payload;
public SingleDocsAndPositionsEnum() {
payload = new BytesRef(8);
payload.length = 8;
}
/** For reuse */
public void reset(int singleDocID, long version, Bits liveDocs) {
doc = -1;
this.liveDocs = liveDocs;
this.singleDocID = singleDocID;
this.version = version;
}
@Override
public int nextDoc() {
if (doc == -1 && (liveDocs == null || liveDocs.get(singleDocID))) {
doc = singleDocID;
} else {
doc = NO_MORE_DOCS;
}
pos = -1;
return doc;
}
@Override
public int docID() {
return doc;
}
@Override
public int advance(int target) {
if (doc == -1 && target <= singleDocID && (liveDocs == null || liveDocs.get(singleDocID))) {
doc = singleDocID;
pos = -1;
} else {
doc = NO_MORE_DOCS;
}
return doc;
}
@Override
public long cost() {
return 1;
}
@Override
public int freq() {
return 1;
}
@Override
public int nextPosition() {
assert pos == -1;
pos = 0;
IDVersionPostingsFormat.longToBytes(version, payload);
return pos;
}
@Override
public BytesRef getPayload() {
return payload;
}
@Override
public int startOffset() {
return -1;
}
@Override
public int endOffset() {
return -1;
}
}

View File

@ -0,0 +1,71 @@
package org.apache.lucene.codecs.idversion;
/*
* 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;
import org.apache.lucene.util.Bits;
class SingleDocsEnum extends DocsEnum {
private int doc;
private int singleDocID;
private Bits liveDocs;
/** For reuse */
public void reset(int singleDocID, Bits liveDocs) {
doc = -1;
this.liveDocs = liveDocs;
this.singleDocID = singleDocID;
}
@Override
public int nextDoc() {
if (doc == -1 && (liveDocs == null || liveDocs.get(singleDocID))) {
doc = singleDocID;
} else {
doc = NO_MORE_DOCS;
}
return doc;
}
@Override
public int docID() {
return doc;
}
@Override
public int advance(int target) {
if (doc == -1 && target <= singleDocID && (liveDocs == null || liveDocs.get(singleDocID))) {
doc = singleDocID;
} else {
doc = NO_MORE_DOCS;
}
return doc;
}
@Override
public long cost() {
return 1;
}
@Override
public int freq() {
return 1;
}
}

View File

@ -0,0 +1,274 @@
package org.apache.lucene.codecs.idversion;
/*
* 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.PrintStream;
import java.util.Collections;
import java.util.Iterator;
import java.util.TreeMap;
import org.apache.lucene.codecs.BlockTermState;
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.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.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;
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.IOUtils;
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.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.PairOutputs.Pair;
import org.apache.lucene.util.fst.PairOutputs;
import org.apache.lucene.util.fst.Util;
/**
* See {@link VersionBlockTreeTermsWriter}.
*
* @lucene.experimental
*/
final class VersionBlockTreeTermsReader extends FieldsProducer {
// Open input to the main terms dict file (_X.tiv)
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,VersionFieldReader> 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 VersionBlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
PostingsReaderBase postingsReader, IOContext ioContext,
String segmentSuffix)
throws IOException {
this.postingsReader = postingsReader;
this.segment = info.name;
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VersionBlockTreeTermsWriter.TERMS_EXTENSION),
ioContext);
boolean success = false;
IndexInput indexIn = null;
try {
version = readHeader(in);
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VersionBlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
ioContext);
int indexVersion = readIndexHeader(indexIn);
if (indexVersion != version) {
throw new CorruptIndexException("mixmatched version files: " + in + "=" + version + "," + indexIn + "=" + indexVersion);
}
// verify
CodecUtil.checksumEntireFile(indexIn);
// Have PostingsReader init itself
postingsReader.init(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 + " (resource=" + in + ")");
}
for(int i=0;i<numFields;i++) {
final int field = in.readVInt();
final long numTerms = in.readVLong();
assert numTerms >= 0;
final int numBytes = in.readVInt();
final BytesRef code = new BytesRef(new byte[numBytes]);
in.readBytes(code.bytes, 0, numBytes);
code.length = numBytes;
final long version = in.readVLong();
final Pair<BytesRef,Long> rootCode = VersionBlockTreeTermsWriter.FST_OUTPUTS.newPair(code, version);
final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
assert fieldInfo != null: "field=" + field;
final long sumTotalTermFreq = numTerms;
final long sumDocFreq = numTerms;
assert numTerms <= Integer.MAX_VALUE;
final int docCount = (int) numTerms;
final int longsSize = in.readVInt();
BytesRef minTerm = readBytesRef(in);
BytesRef maxTerm = readBytesRef(in);
if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs
throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount() + " (resource=" + in + ")");
}
if (sumDocFreq < docCount) { // #postings must be >= #docs with field
throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount + " (resource=" + in + ")");
}
if (sumTotalTermFreq != -1 && sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")");
}
final long indexStartFP = indexIn.readVLong();
VersionFieldReader previous = fields.put(fieldInfo.name,
new VersionFieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount,
indexStartFP, longsSize, indexIn, minTerm, maxTerm));
if (previous != null) {
throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + 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, VersionBlockTreeTermsWriter.TERMS_CODEC_NAME,
VersionBlockTreeTermsWriter.VERSION_START,
VersionBlockTreeTermsWriter.VERSION_CURRENT);
return version;
}
/** Reads index file header. */
private int readIndexHeader(IndexInput input) throws IOException {
int version = CodecUtil.checkHeader(input, VersionBlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
VersionBlockTreeTermsWriter.VERSION_START,
VersionBlockTreeTermsWriter.VERSION_CURRENT);
return version;
}
/** Seek {@code input} to the directory offset. */
private void seekDir(IndexInput input, long dirOffset)
throws IOException {
input.seek(input.length() - CodecUtil.footerLength() - 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 sizeInByes = ((postingsReader!=null) ? postingsReader.ramBytesUsed() : 0);
for(VersionFieldReader reader : fields.values()) {
sizeInByes += reader.ramBytesUsed();
}
return sizeInByes;
}
@Override
public void checkIntegrity() throws IOException {
// term dictionary
CodecUtil.checksumEntireFile(in);
// postings
postingsReader.checkIntegrity();
}
}

View File

@ -0,0 +1,163 @@
package org.apache.lucene.codecs.idversion;
/*
* 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.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
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.BytesRef;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PairOutputs.Pair;
/** BlockTree's implementation of {@link Terms}. */
// public for CheckIndex:
final class VersionFieldReader extends Terms {
final long numTerms;
final FieldInfo fieldInfo;
final long sumTotalTermFreq;
final long sumDocFreq;
final int docCount;
final long indexStartFP;
final long rootBlockFP;
final Pair<BytesRef,Long> rootCode;
final BytesRef minTerm;
final BytesRef maxTerm;
final int longsSize;
final VersionBlockTreeTermsReader parent;
final FST<Pair<BytesRef,Long>> index;
//private boolean DEBUG;
VersionFieldReader(VersionBlockTreeTermsReader parent, FieldInfo fieldInfo, long numTerms, Pair<BytesRef,Long> 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.output1.bytes, rootCode.output1.offset, rootCode.output1.length)).readVLong() >>> VersionBlockTreeTermsWriter.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, VersionBlockTreeTermsWriter.FST_OUTPUTS);
/*
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;
}
}
@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 IDVersionSegmentTermsEnum(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;
}
/** Returns approximate RAM bytes used */
public long ramBytesUsed() {
return ((index!=null)? index.sizeInBytes() : 0);
}
}

View File

@ -0,0 +1,26 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
A primary-key postings format that associates a version (long) with each term and
can provide fail-fast lookups by ID and version.
</body>
</html>

View File

@ -0,0 +1,16 @@
# 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.
org.apache.lucene.codecs.idversion.IDVersionPostingsFormat

View File

@ -0,0 +1,104 @@
package org.apache.lucene.codecs.idversion;
/*
* 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.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.util.BytesRef;
// TODO: can we take a BytesRef token instead?
/** Produces a single String token from the provided value, with the provided payload. */
class StringAndPayloadField extends Field {
public static final FieldType TYPE = new FieldType();
static {
TYPE.setIndexed(true);
TYPE.setOmitNorms(true);
TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
TYPE.setTokenized(true);
TYPE.freeze();
}
private final BytesRef payload;
public StringAndPayloadField(String name, String value, BytesRef payload) {
super(name, value, TYPE);
this.payload = payload;
}
@Override
public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) throws IOException {
SingleTokenWithPayloadTokenStream ts;
if (reuse instanceof SingleTokenWithPayloadTokenStream) {
ts = (SingleTokenWithPayloadTokenStream) reuse;
} else {
ts = new SingleTokenWithPayloadTokenStream();
}
ts.setValue((String) fieldsData, payload);
return ts;
}
static final class SingleTokenWithPayloadTokenStream extends TokenStream {
private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class);
private final PayloadAttribute payloadAttribute = addAttribute(PayloadAttribute.class);
private boolean used = false;
private String value = null;
private BytesRef payload;
/** Sets the string value. */
void setValue(String value, BytesRef payload) {
this.value = value;
this.payload = payload;
}
@Override
public boolean incrementToken() {
if (used) {
return false;
}
clearAttributes();
termAttribute.append(value);
payloadAttribute.setPayload(payload);
used = true;
return true;
}
@Override
public void reset() {
used = false;
}
@Override
public void close() {
value = null;
payload = null;
}
}
}

View File

@ -0,0 +1,841 @@
package org.apache.lucene.codecs.idversion;
/*
* 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.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.analysis.Analyzer.TokenStreamComponents;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenFilter;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.codecs.idversion.StringAndPayloadField.SingleTokenWithPayloadTokenStream;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MergeScheduler;
import org.apache.lucene.index.PerThreadPKLookup;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.LiveFieldValues;
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
/**
* Basic tests for IDVersionPostingsFormat
*/
// Cannot extend BasePostingsFormatTestCase because this PF is not
// general (it requires payloads, only allows 1 doc per term, etc.)
public class TestIDVersionPostingsFormat extends LuceneTestCase {
public void testBasic() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(makeIDField("id0", 100));
w.addDocument(doc);
doc = new Document();
doc.add(makeIDField("id1", 110));
w.addDocument(doc);
IndexReader r = w.getReader();
IDVersionSegmentTermsEnum termsEnum = (IDVersionSegmentTermsEnum) r.leaves().get(0).reader().fields().terms("id").iterator(null);
assertTrue(termsEnum.seekExact(new BytesRef("id0"), 50));
assertTrue(termsEnum.seekExact(new BytesRef("id0"), 100));
assertFalse(termsEnum.seekExact(new BytesRef("id0"), 101));
assertTrue(termsEnum.seekExact(new BytesRef("id1"), 50));
assertTrue(termsEnum.seekExact(new BytesRef("id1"), 110));
assertFalse(termsEnum.seekExact(new BytesRef("id1"), 111));
r.close();
w.close();
dir.close();
}
private interface IDSource {
String next();
}
private IDSource getRandomIDs() {
IDSource ids;
switch (random().nextInt(6)) {
case 0:
// random simple
if (VERBOSE) {
System.out.println("TEST: use random simple ids");
}
ids = new IDSource() {
@Override
public String next() {
return TestUtil.randomSimpleString(random());
}
};
break;
case 1:
// random realistic unicode
if (VERBOSE) {
System.out.println("TEST: use random realistic unicode ids");
}
ids = new IDSource() {
@Override
public String next() {
return TestUtil.randomRealisticUnicodeString(random());
}
};
break;
case 2:
// sequential
if (VERBOSE) {
System.out.println("TEST: use seuquential ids");
}
ids = new IDSource() {
int upto;
@Override
public String next() {
return Integer.toString(upto++);
}
};
break;
case 3:
// zero-pad sequential
if (VERBOSE) {
System.out.println("TEST: use zero-pad seuquential ids");
}
ids = new IDSource() {
final int radix = TestUtil.nextInt(random(), Character.MIN_RADIX, Character.MAX_RADIX);
final String zeroPad = String.format(Locale.ROOT, "%0" + TestUtil.nextInt(random(), 4, 20) + "d", 0);
int upto;
@Override
public String next() {
String s = Integer.toString(upto++);
return zeroPad.substring(zeroPad.length() - s.length()) + s;
}
};
break;
case 4:
// random long
if (VERBOSE) {
System.out.println("TEST: use random long ids");
}
ids = new IDSource() {
final int radix = TestUtil.nextInt(random(), Character.MIN_RADIX, Character.MAX_RADIX);
int upto;
@Override
public String next() {
return Long.toString(random().nextLong() & 0x3ffffffffffffffL, radix);
}
};
break;
case 5:
// zero-pad random long
if (VERBOSE) {
System.out.println("TEST: use zero-pad random long ids");
}
ids = new IDSource() {
final int radix = TestUtil.nextInt(random(), Character.MIN_RADIX, Character.MAX_RADIX);
final String zeroPad = String.format(Locale.ROOT, "%015d", 0);
int upto;
@Override
public String next() {
return Long.toString(random().nextLong() & 0x3ffffffffffffffL, radix);
}
};
break;
default:
throw new AssertionError();
}
return ids;
}
// TODO make a similar test for BT, w/ varied IDs:
public void testRandom() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
int minItemsInBlock = TestUtil.nextInt(random(), 2, 50);
int maxItemsInBlock = 2*(minItemsInBlock-1) + random().nextInt(50);
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat(minItemsInBlock, maxItemsInBlock)));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
//IndexWriter w = new IndexWriter(dir, iwc);
int numDocs = atLeast(1000);
Map<String,Long> idValues = new HashMap<String,Long>();
int docUpto = 0;
if (VERBOSE) {
System.out.println("TEST: numDocs=" + numDocs);
}
IDSource ids = getRandomIDs();
String idPrefix;
if (random().nextBoolean()) {
idPrefix = "";
} else {
idPrefix = TestUtil.randomSimpleString(random());
if (VERBOSE) {
System.out.println("TEST: use id prefix: " + idPrefix);
}
}
boolean useMonotonicVersion = random().nextBoolean();
if (VERBOSE) {
System.out.println("TEST: useMonotonicVersion=" + useMonotonicVersion);
}
List<String> idsList = new ArrayList<>();
long version = 0;
while (docUpto < numDocs) {
String idValue = idPrefix + ids.next();
if (idValues.containsKey(idValue)) {
continue;
}
if (useMonotonicVersion) {
version += TestUtil.nextInt(random(), 1, 10);
} else {
version = random().nextLong() & 0x3fffffffffffffffL;
}
idValues.put(idValue, version);
if (VERBOSE) {
System.out.println(" " + idValue + " -> " + version);
}
Document doc = new Document();
doc.add(makeIDField(idValue, version));
w.addDocument(doc);
idsList.add(idValue);
if (idsList.size() > 0 && random().nextInt(7) == 5) {
// Randomly delete or update a previous ID
idValue = idsList.get(random().nextInt(idsList.size()));
if (random().nextBoolean()) {
if (useMonotonicVersion) {
version += TestUtil.nextInt(random(), 1, 10);
} else {
version = random().nextLong() & 0x3fffffffffffffffL;
}
doc = new Document();
doc.add(makeIDField(idValue, version));
if (VERBOSE) {
System.out.println(" update " + idValue + " -> " + version);
}
w.updateDocument(new Term("id", idValue), doc);
idValues.put(idValue, version);
} else {
if (VERBOSE) {
System.out.println(" delete " + idValue);
}
w.deleteDocuments(new Term("id", idValue));
idValues.remove(idValue);
}
}
docUpto++;
}
IndexReader r = w.getReader();
//IndexReader r = DirectoryReader.open(w, true);
PerThreadVersionPKLookup lookup = new PerThreadVersionPKLookup(r, "id");
List<Map.Entry<String,Long>> idValuesList = new ArrayList<>(idValues.entrySet());
int iters = numDocs * 5;
for(int iter=0;iter<iters;iter++) {
String idValue;
if (random().nextBoolean()) {
idValue = idValuesList.get(random().nextInt(idValuesList.size())).getKey();
} else if (random().nextBoolean()) {
idValue = ids.next();
} else {
idValue = idPrefix + TestUtil.randomSimpleString(random());
}
BytesRef idValueBytes = new BytesRef(idValue);
Long expectedVersion = idValues.get(idValue);
if (VERBOSE) {
System.out.println("\nTEST: iter=" + iter + " id=" + idValue + " expectedVersion=" + expectedVersion);
}
if (expectedVersion == null) {
assertEquals("term should not have been found (doesn't exist)", -1, lookup.lookup(idValueBytes));
} else {
if (random().nextBoolean()) {
if (VERBOSE) {
System.out.println(" lookup exact version (should be found)");
}
assertTrue("term should have been found (version too old)", lookup.lookup(idValueBytes, expectedVersion.longValue()) != -1);
assertEquals(expectedVersion.longValue(), lookup.getVersion());
} else {
if (VERBOSE) {
System.out.println(" lookup version+1 (should not be found)");
}
assertEquals("term should not have been found (version newer)", -1, lookup.lookup(idValueBytes, expectedVersion.longValue()+1));
}
}
}
r.close();
w.close();
dir.close();
}
private static class PerThreadVersionPKLookup extends PerThreadPKLookup {
public PerThreadVersionPKLookup(IndexReader r, String field) throws IOException {
super(r, field);
}
long lastVersion;
/** Returns docID if found, else -1. */
public int lookup(BytesRef id, long version) throws IOException {
for(int seg=0;seg<numSegs;seg++) {
if (((IDVersionSegmentTermsEnum) termsEnums[seg]).seekExact(id, version)) {
if (VERBOSE) {
System.out.println(" found in seg=" + termsEnums[seg]);
}
docsEnums[seg] = termsEnums[seg].docs(liveDocs[seg], docsEnums[seg], 0);
int docID = docsEnums[seg].nextDoc();
if (docID != DocsEnum.NO_MORE_DOCS) {
lastVersion = ((IDVersionSegmentTermsEnum) termsEnums[seg]).getVersion();
return docBases[seg] + docID;
}
assert hasDeletions;
}
}
return -1;
}
/** Only valid if lookup returned a valid docID. */
public long getVersion() {
return lastVersion;
}
}
private static Field makeIDField(String id, long version) {
BytesRef payload = new BytesRef(8);
payload.length = 8;
IDVersionPostingsFormat.longToBytes(version, payload);
return new StringAndPayloadField("id", id, payload);
}
public void testMoreThanOneDocPerIDOneSegment() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(makeIDField("id", 17));
w.addDocument(doc);
doc = new Document();
doc.add(makeIDField("id", 17));
try {
w.addDocument(doc);
w.commit();
fail("didn't hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
}
public void testMoreThanOneDocPerIDTwoSegments() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
iwc.setMergePolicy(new TieredMergePolicy());
MergeScheduler ms = iwc.getMergeScheduler();
if (ms instanceof ConcurrentMergeScheduler) {
iwc.setMergeScheduler(new ConcurrentMergeScheduler() {
@Override
protected void handleMergeException(Throwable exc) {
assertTrue(exc instanceof IllegalArgumentException);
}
});
}
IndexWriter w = new IndexWriter(dir, iwc);
Document doc = new Document();
doc.add(makeIDField("id", 17));
w.addDocument(doc);
w.commit();
doc = new Document();
doc.add(makeIDField("id", 17));
try {
w.addDocument(doc);
w.commit();
w.forceMerge(1);
fail("didn't hit exception");
} catch (IllegalArgumentException iae) {
// expected: SMS will hit this
} catch (IOException ioe) {
// expected
assertTrue(ioe.getCause() instanceof IllegalArgumentException);
}
w.close();
dir.close();
}
public void testMoreThanOneDocPerIDWithUpdates() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(makeIDField("id", 17));
w.addDocument(doc);
doc = new Document();
doc.add(makeIDField("id", 17));
// Replaces the doc we just indexed:
w.updateDocument(new Term("id", "id"), doc);
w.commit();
w.close();
dir.close();
}
public void testMoreThanOneDocPerIDWithDeletes() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(makeIDField("id", 17));
w.addDocument(doc);
w.deleteDocuments(new Term("id", "id"));
doc = new Document();
doc.add(makeIDField("id", 17));
w.addDocument(doc);
w.commit();
w.close();
dir.close();
}
public void testMissingPayload() throws Exception {
Directory dir = newDirectory();
// MockAnalyzer minus maybePayload else it sometimes stuffs in an 8-byte payload!
Analyzer a = new Analyzer() {
@Override
public TokenStreamComponents createComponents(String fieldName) {
MockTokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, true, 100);
tokenizer.setEnableChecks(true);
MockTokenFilter filt = new MockTokenFilter(tokenizer, MockTokenFilter.EMPTY_STOPSET);
return new TokenStreamComponents(tokenizer, filt);
}
};
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, a);
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(newTextField("id", "id", Field.Store.NO));
try {
w.addDocument(doc);
w.commit();
fail("didn't hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
}
public void testMissingPositions() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(newStringField("id", "id", Field.Store.NO));
try {
w.addDocument(doc);
w.commit();
fail("didn't hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
}
public void testInvalidPayload() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(new StringAndPayloadField("id", "id", new BytesRef("foo")));
try {
w.addDocument(doc);
w.commit();
fail("didn't hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
}
public void testMoreThanOneDocPerIDWithDeletesAcrossSegments() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(makeIDField("id", 17));
w.addDocument(doc);
w.commit();
doc = new Document();
doc.add(makeIDField("id", 17));
// Replaces the doc we just indexed:
w.updateDocument(new Term("id", "id"), doc);
w.forceMerge(1);
w.close();
dir.close();
}
// LUCENE-5693: because CheckIndex cross-checks term vectors with postings even for deleted docs, and because our PF only indexes the
// non-deleted documents on flush, CheckIndex will see this as corruption:
public void testCannotIndexTermVectors() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
FieldType ft = new FieldType(StringAndPayloadField.TYPE);
ft.setStoreTermVectors(true);
SingleTokenWithPayloadTokenStream ts = new SingleTokenWithPayloadTokenStream();
BytesRef payload = new BytesRef(8);
payload.length = 8;
IDVersionPostingsFormat.longToBytes(17, payload);
ts.setValue("foo", payload);
Field field = new Field("id", ts, ft);
doc.add(new Field("id", ts, ft));
try {
w.addDocument(doc);
w.commit();
fail("didn't hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
// iae.printStackTrace(System.out);
}
w.close();
dir.close();
}
public void testMoreThanOnceInSingleDoc() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
doc.add(makeIDField("id", 17));
doc.add(makeIDField("id", 17));
try {
w.addDocument(doc);
w.commit();
fail("didn't hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
}
public void testInvalidVersions() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
// -1
doc.add(new StringAndPayloadField("id", "id", new BytesRef(new byte[] {(byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff})));
try {
w.addDocument(doc);
w.commit();
fail("didn't hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
}
doc = new Document();
// Long.MAX_VALUE:
doc.add(new StringAndPayloadField("id", "id", new BytesRef(new byte[] {(byte)0x7f, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff, (byte)0xff})));
try {
w.addDocument(doc);
w.commit();
fail("didn't hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
}
w.close();
dir.close();
}
// Simulates optimistic concurrency in a distributed indexing app and confirms the latest version always wins:
public void testGlobalVersions() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setCodec(TestUtil.alwaysPostingsFormat(new IDVersionPostingsFormat()));
final RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
IDSource idsSource = getRandomIDs();
int numIDs = atLeast(100);
if (VERBOSE) {
System.out.println("TEST: " + numIDs + " ids");
}
Set<String> idsSeen = new HashSet<String>();
while (idsSeen.size() < numIDs) {
idsSeen.add(idsSource.next());
}
final String[] ids = idsSeen.toArray(new String[numIDs]);
final Object[] locks = new Object[ids.length];
for(int i=0;i<locks.length;i++) {
locks[i] = new Object();
}
final AtomicLong nextVersion = new AtomicLong();
final SearcherManager mgr = new SearcherManager(w.w, true, new SearcherFactory());
final Long missingValue = -1L;
final LiveFieldValues<IndexSearcher,Long> versionValues = new LiveFieldValues<IndexSearcher,Long>(mgr, missingValue) {
@Override
protected Long lookupFromSearcher(IndexSearcher s, String id) {
// TODO: would be cleaner if we could do our PerThreadLookup here instead of "up above":
// We always return missing: the caller then does a lookup against the current reader
return missingValue;
}
};
// Maps to the version the id was lasted indexed with:
final Map<String,Long> truth = new ConcurrentHashMap<>();
final CountDownLatch startingGun = new CountDownLatch(1);
Thread[] threads = new Thread[TestUtil.nextInt(random(), 2, 7)];
final int versionType = random().nextInt(3);
if (VERBOSE) {
if (versionType == 0) {
System.out.println("TEST: use random versions");
} else if (versionType == 1) {
System.out.println("TEST: use monotonic versions");
} else {
System.out.println("TEST: use nanotime versions");
}
}
// Run for 3 sec in normal tests, else 60 seconds for nightly:
final long stopTime = System.currentTimeMillis() + (TEST_NIGHTLY ? 60000 : 3000);
for(int i=0;i<threads.length;i++) {
threads[i] = new Thread() {
@Override
public void run() {
try {
runForReal();
} catch (Exception e) {
throw new RuntimeException(e);
}
}
private void runForReal() throws IOException, InterruptedException {
startingGun.await();
PerThreadVersionPKLookup lookup = null;
IndexReader lookupReader = null;
while (System.currentTimeMillis() < stopTime) {
// Intentionally pull version first, and then sleep/yield, to provoke version conflicts:
long newVersion;
if (versionType == 0) {
// Random:
newVersion = random().nextLong() & 0x3fffffffffffffffL;
} else if (versionType == 1) {
// Monotonic
newVersion = nextVersion.getAndIncrement();
} else {
newVersion = System.nanoTime();
}
if (versionType != 0) {
if (random().nextBoolean()) {
Thread.yield();
} else {
Thread.sleep(TestUtil.nextInt(random(), 1, 4));
}
}
int x = random().nextInt(ids.length);
// TODO: we could relax this, if e.g. we assign indexer thread based on ID. This would ensure a given ID cannot be indexed at
// the same time in multiple threads:
// Only one thread can update an ID at once:
synchronized (locks[x]) {
String id = ids[x];
// We will attempt to index id with newVersion, but only do so if id wasn't yet indexed, or it was indexed with an older
// version (< newVersion):
// Must lookup the RT value before pulling from the index, in case a reopen happens just after we lookup:
Long currentVersion = versionValues.get(id);
IndexSearcher s = mgr.acquire();
try {
if (VERBOSE) System.out.println("\n" + Thread.currentThread().getName() + ": update id=" + id + " newVersion=" + newVersion);
if (lookup == null || lookupReader != s.getIndexReader()) {
// TODO: sort of messy; we could add reopen to PerThreadVersionPKLookup?
// TODO: this is thin ice .... that we don't incRef/decRef this reader we are implicitly holding onto:
lookupReader = s.getIndexReader();
if (VERBOSE) System.out.println(Thread.currentThread().getName() + ": open new PK lookup reader=" + lookupReader);
lookup = new PerThreadVersionPKLookup(lookupReader, "id");
}
Long truthVersion = truth.get(id);
if (VERBOSE) System.out.println(Thread.currentThread().getName() + ": truthVersion=" + truthVersion);
boolean doIndex;
if (currentVersion == missingValue) {
if (VERBOSE) System.out.println(Thread.currentThread().getName() + ": id not in RT cache");
int otherDocID = lookup.lookup(new BytesRef(id), newVersion+1);
if (otherDocID == -1) {
if (VERBOSE) System.out.println(Thread.currentThread().getName() + ": id not in index, or version is <= newVersion; will index");
doIndex = true;
} else {
if (VERBOSE) System.out.println(Thread.currentThread().getName() + ": id is in index with version=" + lookup.getVersion() + "; will not index");
doIndex = false;
if (truthVersion.longValue() !=lookup.getVersion()) {
System.out.println(Thread.currentThread() + ": now fail0!");
}
assertEquals(truthVersion.longValue(), lookup.getVersion());
}
} else {
if (VERBOSE) System.out.println(Thread.currentThread().getName() + ": id is in RT cache: currentVersion=" + currentVersion);
doIndex = newVersion > currentVersion;
}
if (doIndex) {
if (VERBOSE) System.out.println(Thread.currentThread().getName() + ": now index");
boolean passes = truthVersion == null || truthVersion.longValue() <= newVersion;
if (passes == false) {
System.out.println(Thread.currentThread() + ": now fail!");
}
assertTrue(passes);
Document doc = new Document();
doc.add(makeIDField(id, newVersion));
w.updateDocument(new Term("id", id), doc);
truth.put(id, newVersion);
versionValues.add(id, newVersion);
} else {
if (VERBOSE) System.out.println(Thread.currentThread().getName() + ": skip index");
assertNotNull(truthVersion);
assertTrue(truthVersion.longValue() >= newVersion);
}
} finally {
mgr.release(s);
}
}
}
}
};
threads[i].start();
}
startingGun.countDown();
// Keep reopening the NRT reader until all indexing threads are done:
refreshLoop:
while (true) {
Thread.sleep(TestUtil.nextInt(random(), 1, 10));
mgr.maybeRefresh();
for (Thread thread : threads) {
if (thread.isAlive()) {
continue refreshLoop;
}
}
break;
}
// Verify final index against truth:
for(int i=0;i<2;i++) {
mgr.maybeRefresh();
IndexSearcher s = mgr.acquire();
try {
IndexReader r = s.getIndexReader();
// cannot assert this: maybe not all IDs were indexed
/*
assertEquals(numIDs, r.numDocs());
if (i == 1) {
// After forceMerge no deleted docs:
assertEquals(numIDs, r.maxDoc());
}
*/
PerThreadVersionPKLookup lookup = new PerThreadVersionPKLookup(r, "id");
for(Map.Entry<String,Long> ent : truth.entrySet()) {
assertTrue(lookup.lookup(new BytesRef(ent.getKey()), -1L) != -1);
assertEquals(ent.getValue().longValue(), lookup.getVersion());
}
} finally {
mgr.release(s);
}
if (i == 1) {
break;
}
// forceMerge and verify again
w.forceMerge(1);
}
mgr.close();
w.close();
dir.close();
}
}

View File

@ -19,9 +19,9 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.LuceneTestCase;

View File

@ -20,8 +20,6 @@ package org.apache.lucene.codecs.mockrandom;
import java.io.IOException;
import java.util.Random;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
@ -36,6 +34,8 @@ import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexReader;
import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.memory.FSTOrdTermsReader;

View File

@ -19,13 +19,13 @@ package org.apache.lucene.codecs.nestedpulsing;
import java.io.IOException;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.BlockTreeTermsWriter;
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.PostingsWriterBase;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
import org.apache.lucene.codecs.pulsing.PulsingPostingsReader;

View File

@ -6,6 +6,7 @@ import java.util.Iterator;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.VirtualMethod;
import org.apache.lucene.util.automaton.CompiledAutomaton;
/*
@ -117,12 +118,16 @@ public class AssertingAtomicReader extends FilterAtomicReader {
}
}
static final VirtualMethod<TermsEnum> SEEK_EXACT = new VirtualMethod<>(TermsEnum.class, "seekExact", BytesRef.class);
static class AssertingTermsEnum extends FilterTermsEnum {
private enum State {INITIAL, POSITIONED, UNPOSITIONED};
private State state = State.INITIAL;
private final boolean delegateOverridesSeekExact;
public AssertingTermsEnum(TermsEnum in) {
super(in);
delegateOverridesSeekExact = SEEK_EXACT.isOverriddenAsOf(in.getClass());
}
@Override
@ -213,13 +218,18 @@ public class AssertingAtomicReader extends FilterAtomicReader {
@Override
public boolean seekExact(BytesRef text) throws IOException {
assert text.isValid();
if (super.seekExact(text)) {
boolean result;
if (delegateOverridesSeekExact) {
result = in.seekExact(text);
} else {
result = super.seekExact(text);
}
if (result) {
state = State.POSITIONED;
return true;
} else {
state = State.UNPOSITIONED;
return false;
}
return result;
}
@Override
@ -234,6 +244,11 @@ public class AssertingAtomicReader extends FilterAtomicReader {
super.seekExact(term, state);
this.state = State.POSITIONED;
}
@Override
public String toString() {
return "AssertingTermsEnum(" + in + ")";
}
}
static enum DocsEnumState { START, ITERATING, FINISHED };
@ -682,4 +697,4 @@ public class AssertingAtomicReader extends FilterAtomicReader {
}
private final Object cacheKey = new Object();
}
}

View File

@ -0,0 +1,97 @@
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.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
/** Utility class to do efficient primary-key (only 1 doc contains the
* given term) lookups by segment, re-using the enums. This class is
* not thread safe, so it is the caller's job to create and use one
* instance of this per thread. Do not use this if a term may appear
* in more than one document! It will only return the first one it
* finds. */
public class PerThreadPKLookup {
protected final TermsEnum[] termsEnums;
protected final DocsEnum[] docsEnums;
protected final Bits[] liveDocs;
protected final int[] docBases;
protected final int numSegs;
protected final boolean hasDeletions;
public PerThreadPKLookup(IndexReader r, String idFieldName) throws IOException {
List<AtomicReaderContext> leaves = new ArrayList<>(r.leaves());
// Larger segments are more likely to have the id, so we sort largest to smallest by numDocs:
Collections.sort(leaves, new Comparator<AtomicReaderContext>() {
@Override
public int compare(AtomicReaderContext c1, AtomicReaderContext c2) {
return c2.reader().numDocs() - c1.reader().numDocs();
}
});
termsEnums = new TermsEnum[leaves.size()];
docsEnums = new DocsEnum[leaves.size()];
liveDocs = new Bits[leaves.size()];
docBases = new int[leaves.size()];
int numSegs = 0;
boolean hasDeletions = false;
for(int i=0;i<leaves.size();i++) {
Fields fields = leaves.get(i).reader().fields();
if (fields != null) {
Terms terms = fields.terms(idFieldName);
if (terms != null) {
termsEnums[numSegs] = terms.iterator(null);
assert termsEnums[numSegs] != null;
docBases[numSegs] = leaves.get(i).docBase;
liveDocs[numSegs] = leaves.get(i).reader().getLiveDocs();
hasDeletions |= leaves.get(i).reader().hasDeletions();
numSegs++;
}
}
}
this.numSegs = numSegs;
this.hasDeletions = hasDeletions;
}
/** Returns docID if found, else -1. */
public int lookup(BytesRef id) throws IOException {
for(int seg=0;seg<numSegs;seg++) {
if (termsEnums[seg].seekExact(id)) {
docsEnums[seg] = termsEnums[seg].docs(liveDocs[seg], docsEnums[seg], 0);
int docID = docsEnums[seg].nextDoc();
if (docID != DocsEnum.NO_MORE_DOCS) {
return docBases[seg] + docID;
}
assert hasDeletions;
}
}
return -1;
}
// TODO: add reopen method to carry over re-used enums...?
}

View File

@ -31,8 +31,8 @@ import org.apache.lucene.util.VirtualMethod;
/** Wraps a Scorer with additional checks */
public class AssertingBulkScorer extends BulkScorer {
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", LeafCollector.class);
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", LeafCollector.class, int.class);
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR = new VirtualMethod<>(BulkScorer.class, "score", LeafCollector.class);
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<>(BulkScorer.class, "score", LeafCollector.class, int.class);
public static BulkScorer wrap(Random random, BulkScorer other) {
if (other == null || other instanceof AssertingBulkScorer) {