mirror of https://github.com/apache/lucene.git
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:
commit
8a1587d6d8
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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>
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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<>();
|
||||
|
|
|
@ -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++;
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -105,5 +105,6 @@ public class SegmentWriteState {
|
|||
this.segmentSuffix = segmentSuffix;
|
||||
segUpdates = state.segUpdates;
|
||||
delCountOnFlush = state.delCountOnFlush;
|
||||
liveDocs = state.liveDocs;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 {
|
||||
}
|
||||
}
|
|
@ -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 {
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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>
|
|
@ -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
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 };
|
||||
|
|
|
@ -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...?
|
||||
}
|
|
@ -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) {
|
||||
|
|
Loading…
Reference in New Issue