mirror of https://github.com/apache/lucene.git
LUCENE-3069: API refactoring on BlockTerms dict
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3069@1514253 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
134cfe0b29
commit
dd1070cd2b
|
@ -0,0 +1,142 @@
|
|||
package org.apache.lucene.codecs.temp;
|
||||
|
||||
/*
|
||||
* 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.TempPostingsReaderBase;
|
||||
import org.apache.lucene.codecs.TempPostingsWriterBase;
|
||||
import org.apache.lucene.codecs.blockterms.BlockTermsReader;
|
||||
import org.apache.lucene.codecs.blockterms.BlockTermsWriter;
|
||||
import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexReader;
|
||||
import org.apache.lucene.codecs.blockterms.FixedGapTermsIndexWriter;
|
||||
import org.apache.lucene.codecs.blockterms.TermsIndexReaderBase;
|
||||
import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
// TODO: we could make separate base class that can wrapp
|
||||
// any PostingsBaseFormat and make it ord-able...
|
||||
|
||||
/**
|
||||
* Customized version of {@link Lucene41PostingsFormat} that uses
|
||||
* {@link FixedGapTermsIndexWriter}.
|
||||
*/
|
||||
public final class TempBlockPostingsFormat extends PostingsFormat {
|
||||
final int termIndexInterval;
|
||||
|
||||
public TempBlockPostingsFormat() {
|
||||
this(FixedGapTermsIndexWriter.DEFAULT_TERM_INDEX_INTERVAL);
|
||||
}
|
||||
|
||||
public TempBlockPostingsFormat(int termIndexInterval) {
|
||||
super("TempBlock");
|
||||
this.termIndexInterval = termIndexInterval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
TempPostingsWriterBase docs = new TempPostingsWriter(state);
|
||||
|
||||
// TODO: should we make the terms index more easily
|
||||
// pluggable? Ie so that this codec would record which
|
||||
// index impl was used, and switch on loading?
|
||||
// Or... you must make a new Codec for this?
|
||||
TermsIndexWriterBase indexWriter;
|
||||
boolean success = false;
|
||||
try {
|
||||
indexWriter = new FixedGapTermsIndexWriter(state, termIndexInterval);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
docs.close();
|
||||
}
|
||||
}
|
||||
|
||||
success = false;
|
||||
try {
|
||||
// Must use BlockTermsWriter (not BlockTree) because
|
||||
// BlockTree doens't support ords (yet)...
|
||||
FieldsConsumer ret = new TempBlockTermsWriter(indexWriter, state, docs);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
docs.close();
|
||||
} finally {
|
||||
indexWriter.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
TempPostingsReaderBase postings = new TempPostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
|
||||
TermsIndexReaderBase indexReader;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
indexReader = new FixedGapTermsIndexReader(state.directory,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo.name,
|
||||
BytesRef.getUTF8SortedAsUnicodeComparator(),
|
||||
state.segmentSuffix, state.context);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
postings.close();
|
||||
}
|
||||
}
|
||||
|
||||
success = false;
|
||||
try {
|
||||
FieldsProducer ret = new TempBlockTermsReader(indexReader,
|
||||
state.directory,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo,
|
||||
postings,
|
||||
state.context,
|
||||
state.segmentSuffix);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
postings.close();
|
||||
} finally {
|
||||
indexReader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Extension of freq postings file */
|
||||
static final String FREQ_EXTENSION = "frq";
|
||||
|
||||
/** Extension of prox postings file */
|
||||
static final String PROX_EXTENSION = "prx";
|
||||
}
|
|
@ -0,0 +1,872 @@
|
|||
package org.apache.lucene.codecs.temp;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.TreeMap;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.TempPostingsReaderBase;
|
||||
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.DoubleBarrelLRUCache;
|
||||
import org.apache.lucene.codecs.blockterms.*;
|
||||
|
||||
/** Handles a terms dict, but decouples all details of
|
||||
* doc/freqs/positions reading to an instance of {@link
|
||||
* TempPostingsReaderBase}. This class is reusable for
|
||||
* codecs that use a different format for
|
||||
* docs/freqs/positions (though codecs are also free to
|
||||
* make their own terms dict impl).
|
||||
*
|
||||
* <p>This class also interacts with an instance of {@link
|
||||
* TermsIndexReaderBase}, to abstract away the specific
|
||||
* implementation of the terms dict index.
|
||||
* @lucene.experimental */
|
||||
|
||||
public class TempBlockTermsReader extends FieldsProducer {
|
||||
// Open input to the main terms dict file (_X.tis)
|
||||
private final IndexInput in;
|
||||
|
||||
// Reads the terms dict entries, to gather state to
|
||||
// produce DocsEnum on demand
|
||||
private final TempPostingsReaderBase postingsReader;
|
||||
|
||||
private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
|
||||
|
||||
// Reads the terms index
|
||||
private TermsIndexReaderBase indexReader;
|
||||
|
||||
// keeps the dirStart offset
|
||||
private long dirOffset;
|
||||
|
||||
private final int version;
|
||||
|
||||
// Used as key for the terms cache
|
||||
private static class FieldAndTerm extends DoubleBarrelLRUCache.CloneableKey {
|
||||
String field;
|
||||
BytesRef term;
|
||||
|
||||
public FieldAndTerm() {
|
||||
}
|
||||
|
||||
public FieldAndTerm(FieldAndTerm other) {
|
||||
field = other.field;
|
||||
term = BytesRef.deepCopyOf(other.term);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object _other) {
|
||||
FieldAndTerm other = (FieldAndTerm) _other;
|
||||
return other.field.equals(field) && term.bytesEquals(other.term);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldAndTerm clone() {
|
||||
return new FieldAndTerm(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return field.hashCode() * 31 + term.hashCode();
|
||||
}
|
||||
}
|
||||
|
||||
// private String segment;
|
||||
|
||||
public TempBlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, SegmentInfo info, TempPostingsReaderBase postingsReader, IOContext context,
|
||||
String segmentSuffix)
|
||||
throws IOException {
|
||||
|
||||
this.postingsReader = postingsReader;
|
||||
|
||||
// this.segment = segment;
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(info.name, segmentSuffix, TempBlockTermsWriter.TERMS_EXTENSION),
|
||||
context);
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
version = readHeader(in);
|
||||
|
||||
// Have PostingsReader init itself
|
||||
postingsReader.init(in);
|
||||
|
||||
// Read per-field details
|
||||
seekDir(in, dirOffset);
|
||||
|
||||
final int numFields = in.readVInt();
|
||||
if (numFields < 0) {
|
||||
throw new CorruptIndexException("invalid number of fields: " + numFields + " (resource=" + in + ")");
|
||||
}
|
||||
for(int i=0;i<numFields;i++) {
|
||||
final int field = in.readVInt();
|
||||
final long numTerms = in.readVLong();
|
||||
assert numTerms >= 0;
|
||||
final long termsStartPointer = in.readVLong();
|
||||
final FieldInfo fieldInfo = fieldInfos.fieldInfo(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 = in.readVInt();
|
||||
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 + ")");
|
||||
}
|
||||
FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, termsStartPointer, sumTotalTermFreq, sumDocFreq, docCount, longsSize));
|
||||
if (previous != null) {
|
||||
throw new CorruptIndexException("duplicate fields: " + fieldInfo.name + " (resource=" + in + ")");
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
|
||||
this.indexReader = indexReader;
|
||||
}
|
||||
|
||||
private int readHeader(IndexInput input) throws IOException {
|
||||
int version = CodecUtil.checkHeader(input, TempBlockTermsWriter.CODEC_NAME,
|
||||
TempBlockTermsWriter.VERSION_START,
|
||||
TempBlockTermsWriter.VERSION_CURRENT);
|
||||
if (version < TempBlockTermsWriter.VERSION_APPEND_ONLY) {
|
||||
dirOffset = input.readLong();
|
||||
}
|
||||
return version;
|
||||
}
|
||||
|
||||
private void seekDir(IndexInput input, long dirOffset) throws IOException {
|
||||
if (version >= TempBlockTermsWriter.VERSION_APPEND_ONLY) {
|
||||
input.seek(input.length() - 8);
|
||||
dirOffset = input.readLong();
|
||||
}
|
||||
input.seek(dirOffset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
try {
|
||||
if (indexReader != null) {
|
||||
indexReader.close();
|
||||
}
|
||||
} finally {
|
||||
// null so if an app hangs on to us (ie, we are not
|
||||
// GCable, despite being closed) we still free most
|
||||
// ram
|
||||
indexReader = null;
|
||||
if (in != null) {
|
||||
in.close();
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (postingsReader != null) {
|
||||
postingsReader.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@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();
|
||||
}
|
||||
|
||||
private class FieldReader extends Terms {
|
||||
final long numTerms;
|
||||
final FieldInfo fieldInfo;
|
||||
final long termsStartPointer;
|
||||
final long sumTotalTermFreq;
|
||||
final long sumDocFreq;
|
||||
final int docCount;
|
||||
final int longsSize;
|
||||
|
||||
FieldReader(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize) {
|
||||
assert numTerms > 0;
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.numTerms = numTerms;
|
||||
this.termsStartPointer = termsStartPointer;
|
||||
this.sumTotalTermFreq = sumTotalTermFreq;
|
||||
this.sumDocFreq = sumDocFreq;
|
||||
this.docCount = docCount;
|
||||
this.longsSize = longsSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
return new SegmentTermsEnum();
|
||||
}
|
||||
|
||||
@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 long size() {
|
||||
return numTerms;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumTotalTermFreq() {
|
||||
return sumTotalTermFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumDocFreq() throws IOException {
|
||||
return sumDocFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getDocCount() throws IOException {
|
||||
return docCount;
|
||||
}
|
||||
|
||||
// Iterates through terms in this field
|
||||
private final class SegmentTermsEnum extends TermsEnum {
|
||||
private final IndexInput in;
|
||||
private final TempTermState state;
|
||||
private final boolean doOrd;
|
||||
private final FieldAndTerm fieldTerm = new FieldAndTerm();
|
||||
private final TermsIndexReaderBase.FieldIndexEnum indexEnum;
|
||||
private final BytesRef term = new BytesRef();
|
||||
|
||||
/* This is true if indexEnum is "still" seek'd to the index term
|
||||
for the current term. We set it to true on seeking, and then it
|
||||
remains valid until next() is called enough times to load another
|
||||
terms block: */
|
||||
private boolean indexIsCurrent;
|
||||
|
||||
/* True if we've already called .next() on the indexEnum, to "bracket"
|
||||
the current block of terms: */
|
||||
private boolean didIndexNext;
|
||||
|
||||
/* Next index term, bracketing the current block of terms; this is
|
||||
only valid if didIndexNext is true: */
|
||||
private BytesRef nextIndexTerm;
|
||||
|
||||
/* True after seekExact(TermState), do defer seeking. If the app then
|
||||
calls next() (which is not "typical"), then we'll do the real seek */
|
||||
private boolean seekPending;
|
||||
|
||||
private byte[] termSuffixes;
|
||||
private ByteArrayDataInput termSuffixesReader = new ByteArrayDataInput();
|
||||
|
||||
/* Common prefix used for all terms in this block. */
|
||||
private int termBlockPrefix;
|
||||
|
||||
/* How many terms in current block */
|
||||
private int blockTermCount;
|
||||
|
||||
private byte[] docFreqBytes;
|
||||
private final ByteArrayDataInput freqReader = new ByteArrayDataInput();
|
||||
private int metaDataUpto;
|
||||
|
||||
private long[] longs;
|
||||
private byte[] bytes;
|
||||
private ByteArrayDataInput bytesReader;
|
||||
|
||||
|
||||
public SegmentTermsEnum() throws IOException {
|
||||
in = TempBlockTermsReader.this.in.clone();
|
||||
in.seek(termsStartPointer);
|
||||
indexEnum = indexReader.getFieldEnum(fieldInfo);
|
||||
doOrd = indexReader.supportsOrd();
|
||||
fieldTerm.field = fieldInfo.name;
|
||||
state = postingsReader.newTermState();
|
||||
state.totalTermFreq = -1;
|
||||
state.ord = -1;
|
||||
|
||||
termSuffixes = new byte[128];
|
||||
docFreqBytes = new byte[64];
|
||||
//System.out.println("BTR.enum init this=" + this + " postingsReader=" + postingsReader);
|
||||
longs = new long[longsSize];
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
// TODO: we may want an alternate mode here which is
|
||||
// "if you are about to return NOT_FOUND I won't use
|
||||
// the terms data from that"; eg FuzzyTermsEnum will
|
||||
// (usually) just immediately call seek again if we
|
||||
// return NOT_FOUND so it's a waste for us to fill in
|
||||
// the term that was actually NOT_FOUND
|
||||
@Override
|
||||
public SeekStatus seekCeil(final BytesRef target) throws IOException {
|
||||
|
||||
if (indexEnum == null) {
|
||||
throw new IllegalStateException("terms index was not loaded");
|
||||
}
|
||||
|
||||
//System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this=" + this);
|
||||
if (didIndexNext) {
|
||||
if (nextIndexTerm == null) {
|
||||
//System.out.println(" nextIndexTerm=null");
|
||||
} else {
|
||||
//System.out.println(" nextIndexTerm=" + nextIndexTerm.utf8ToString());
|
||||
}
|
||||
}
|
||||
|
||||
boolean doSeek = true;
|
||||
|
||||
// See if we can avoid seeking, because target term
|
||||
// is after current term but before next index term:
|
||||
if (indexIsCurrent) {
|
||||
|
||||
final int cmp = BytesRef.getUTF8SortedAsUnicodeComparator().compare(term, target);
|
||||
|
||||
if (cmp == 0) {
|
||||
// Already at the requested term
|
||||
return SeekStatus.FOUND;
|
||||
} else if (cmp < 0) {
|
||||
|
||||
// Target term is after current term
|
||||
if (!didIndexNext) {
|
||||
if (indexEnum.next() == -1) {
|
||||
nextIndexTerm = null;
|
||||
} else {
|
||||
nextIndexTerm = indexEnum.term();
|
||||
}
|
||||
//System.out.println(" now do index next() nextIndexTerm=" + (nextIndexTerm == null ? "null" : nextIndexTerm.utf8ToString()));
|
||||
didIndexNext = true;
|
||||
}
|
||||
|
||||
if (nextIndexTerm == null || BytesRef.getUTF8SortedAsUnicodeComparator().compare(target, nextIndexTerm) < 0) {
|
||||
// Optimization: requested term is within the
|
||||
// same term block we are now in; skip seeking
|
||||
// (but do scanning):
|
||||
doSeek = false;
|
||||
//System.out.println(" skip seek: nextIndexTerm=" + (nextIndexTerm == null ? "null" : nextIndexTerm.utf8ToString()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (doSeek) {
|
||||
//System.out.println(" seek");
|
||||
|
||||
// Ask terms index to find biggest indexed term (=
|
||||
// first term in a block) that's <= our text:
|
||||
in.seek(indexEnum.seek(target));
|
||||
boolean result = nextBlock();
|
||||
|
||||
// Block must exist since, at least, the indexed term
|
||||
// is in the block:
|
||||
assert result;
|
||||
|
||||
indexIsCurrent = true;
|
||||
didIndexNext = false;
|
||||
|
||||
if (doOrd) {
|
||||
state.ord = indexEnum.ord()-1;
|
||||
}
|
||||
|
||||
term.copyBytes(indexEnum.term());
|
||||
//System.out.println(" seek: term=" + term.utf8ToString());
|
||||
} else {
|
||||
//System.out.println(" skip seek");
|
||||
if (state.termBlockOrd == blockTermCount && !nextBlock()) {
|
||||
indexIsCurrent = false;
|
||||
return SeekStatus.END;
|
||||
}
|
||||
}
|
||||
|
||||
seekPending = false;
|
||||
|
||||
int common = 0;
|
||||
|
||||
// Scan within block. We could do this by calling
|
||||
// _next() and testing the resulting term, but this
|
||||
// is wasteful. Instead, we first confirm the
|
||||
// target matches the common prefix of this block,
|
||||
// and then we scan the term bytes directly from the
|
||||
// termSuffixesreader's byte[], saving a copy into
|
||||
// the BytesRef term per term. Only when we return
|
||||
// do we then copy the bytes into the term.
|
||||
|
||||
while(true) {
|
||||
|
||||
// First, see if target term matches common prefix
|
||||
// in this block:
|
||||
if (common < termBlockPrefix) {
|
||||
final int cmp = (term.bytes[common]&0xFF) - (target.bytes[target.offset + common]&0xFF);
|
||||
if (cmp < 0) {
|
||||
|
||||
// TODO: maybe we should store common prefix
|
||||
// in block header? (instead of relying on
|
||||
// last term of previous block)
|
||||
|
||||
// Target's prefix is after the common block
|
||||
// prefix, so term cannot be in this block
|
||||
// but it could be in next block. We
|
||||
// must scan to end-of-block to set common
|
||||
// prefix for next block:
|
||||
if (state.termBlockOrd < blockTermCount) {
|
||||
while(state.termBlockOrd < blockTermCount-1) {
|
||||
state.termBlockOrd++;
|
||||
state.ord++;
|
||||
termSuffixesReader.skipBytes(termSuffixesReader.readVInt());
|
||||
}
|
||||
final int suffix = termSuffixesReader.readVInt();
|
||||
term.length = termBlockPrefix + suffix;
|
||||
if (term.bytes.length < term.length) {
|
||||
term.grow(term.length);
|
||||
}
|
||||
termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
|
||||
}
|
||||
state.ord++;
|
||||
|
||||
if (!nextBlock()) {
|
||||
indexIsCurrent = false;
|
||||
return SeekStatus.END;
|
||||
}
|
||||
common = 0;
|
||||
|
||||
} else if (cmp > 0) {
|
||||
// Target's prefix is before the common prefix
|
||||
// of this block, so we position to start of
|
||||
// block and return NOT_FOUND:
|
||||
assert state.termBlockOrd == 0;
|
||||
|
||||
final int suffix = termSuffixesReader.readVInt();
|
||||
term.length = termBlockPrefix + suffix;
|
||||
if (term.bytes.length < term.length) {
|
||||
term.grow(term.length);
|
||||
}
|
||||
termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
|
||||
return SeekStatus.NOT_FOUND;
|
||||
} else {
|
||||
common++;
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
// Test every term in this block
|
||||
while (true) {
|
||||
state.termBlockOrd++;
|
||||
state.ord++;
|
||||
|
||||
final int suffix = termSuffixesReader.readVInt();
|
||||
|
||||
// We know the prefix matches, so just compare the new suffix:
|
||||
final int termLen = termBlockPrefix + suffix;
|
||||
int bytePos = termSuffixesReader.getPosition();
|
||||
|
||||
boolean next = false;
|
||||
final int limit = target.offset + (termLen < target.length ? termLen : target.length);
|
||||
int targetPos = target.offset + termBlockPrefix;
|
||||
while(targetPos < limit) {
|
||||
final int cmp = (termSuffixes[bytePos++]&0xFF) - (target.bytes[targetPos++]&0xFF);
|
||||
if (cmp < 0) {
|
||||
// Current term is still before the target;
|
||||
// keep scanning
|
||||
next = true;
|
||||
break;
|
||||
} else if (cmp > 0) {
|
||||
// Done! Current term is after target. Stop
|
||||
// here, fill in real term, return NOT_FOUND.
|
||||
term.length = termBlockPrefix + suffix;
|
||||
if (term.bytes.length < term.length) {
|
||||
term.grow(term.length);
|
||||
}
|
||||
termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
|
||||
//System.out.println(" NOT_FOUND");
|
||||
return SeekStatus.NOT_FOUND;
|
||||
}
|
||||
}
|
||||
|
||||
if (!next && target.length <= termLen) {
|
||||
term.length = termBlockPrefix + suffix;
|
||||
if (term.bytes.length < term.length) {
|
||||
term.grow(term.length);
|
||||
}
|
||||
termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
|
||||
|
||||
if (target.length == termLen) {
|
||||
// Done! Exact match. Stop here, fill in
|
||||
// real term, return FOUND.
|
||||
//System.out.println(" FOUND");
|
||||
return SeekStatus.FOUND;
|
||||
} else {
|
||||
//System.out.println(" NOT_FOUND");
|
||||
return SeekStatus.NOT_FOUND;
|
||||
}
|
||||
}
|
||||
|
||||
if (state.termBlockOrd == blockTermCount) {
|
||||
// Must pre-fill term for next block's common prefix
|
||||
term.length = termBlockPrefix + suffix;
|
||||
if (term.bytes.length < term.length) {
|
||||
term.grow(term.length);
|
||||
}
|
||||
termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
|
||||
break;
|
||||
} else {
|
||||
termSuffixesReader.skipBytes(suffix);
|
||||
}
|
||||
}
|
||||
|
||||
// The purpose of the terms dict index is to seek
|
||||
// the enum to the closest index term before the
|
||||
// term we are looking for. So, we should never
|
||||
// cross another index term (besides the first
|
||||
// one) while we are scanning:
|
||||
|
||||
assert indexIsCurrent;
|
||||
|
||||
if (!nextBlock()) {
|
||||
//System.out.println(" END");
|
||||
indexIsCurrent = false;
|
||||
return SeekStatus.END;
|
||||
}
|
||||
common = 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef next() throws IOException {
|
||||
//System.out.println("BTR.next() seekPending=" + seekPending + " pendingSeekCount=" + state.termBlockOrd);
|
||||
|
||||
// If seek was previously called and the term was cached,
|
||||
// usually caller is just going to pull a D/&PEnum or get
|
||||
// docFreq, etc. But, if they then call next(),
|
||||
// this method catches up all internal state so next()
|
||||
// works properly:
|
||||
if (seekPending) {
|
||||
assert !indexIsCurrent;
|
||||
in.seek(state.blockFilePointer);
|
||||
final int pendingSeekCount = state.termBlockOrd;
|
||||
boolean result = nextBlock();
|
||||
|
||||
final long savOrd = state.ord;
|
||||
|
||||
// Block must exist since seek(TermState) was called w/ a
|
||||
// TermState previously returned by this enum when positioned
|
||||
// on a real term:
|
||||
assert result;
|
||||
|
||||
while(state.termBlockOrd < pendingSeekCount) {
|
||||
BytesRef nextResult = _next();
|
||||
assert nextResult != null;
|
||||
}
|
||||
seekPending = false;
|
||||
state.ord = savOrd;
|
||||
}
|
||||
return _next();
|
||||
}
|
||||
|
||||
/* Decodes only the term bytes of the next term. If caller then asks for
|
||||
metadata, ie docFreq, totalTermFreq or pulls a D/&PEnum, we then (lazily)
|
||||
decode all metadata up to the current term. */
|
||||
private BytesRef _next() throws IOException {
|
||||
//System.out.println("BTR._next seg=" + segment + " this=" + this + " termCount=" + state.termBlockOrd + " (vs " + blockTermCount + ")");
|
||||
if (state.termBlockOrd == blockTermCount && !nextBlock()) {
|
||||
//System.out.println(" eof");
|
||||
indexIsCurrent = false;
|
||||
return null;
|
||||
}
|
||||
|
||||
// TODO: cutover to something better for these ints! simple64?
|
||||
final int suffix = termSuffixesReader.readVInt();
|
||||
//System.out.println(" suffix=" + suffix);
|
||||
|
||||
term.length = termBlockPrefix + suffix;
|
||||
if (term.bytes.length < term.length) {
|
||||
term.grow(term.length);
|
||||
}
|
||||
termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
|
||||
state.termBlockOrd++;
|
||||
|
||||
// NOTE: meaningless in the non-ord case
|
||||
state.ord++;
|
||||
|
||||
//System.out.println(" return term=" + fieldInfo.name + ":" + term.utf8ToString() + " " + term + " tbOrd=" + state.termBlockOrd);
|
||||
return term;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef term() {
|
||||
return term;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq() throws IOException {
|
||||
//System.out.println("BTR.docFreq");
|
||||
decodeMetaData();
|
||||
//System.out.println(" return " + state.docFreq);
|
||||
return state.docFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long totalTermFreq() throws IOException {
|
||||
decodeMetaData();
|
||||
return state.totalTermFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
|
||||
//System.out.println("BTR.docs this=" + this);
|
||||
decodeMetaData();
|
||||
//System.out.println("BTR.docs: state.docFreq=" + state.docFreq);
|
||||
return postingsReader.docs(fieldInfo, state, liveDocs, reuse, flags);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
|
||||
if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
|
||||
// Positions were not indexed:
|
||||
return null;
|
||||
}
|
||||
|
||||
decodeMetaData();
|
||||
return postingsReader.docsAndPositions(fieldInfo, state, liveDocs, reuse, flags);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seekExact(BytesRef target, TermState otherState) {
|
||||
//System.out.println("BTR.seekExact termState target=" + target.utf8ToString() + " " + target + " this=" + this);
|
||||
assert otherState != null && otherState instanceof TempTermState;
|
||||
assert !doOrd || ((TempTermState) otherState).ord < numTerms;
|
||||
state.copyFrom(otherState);
|
||||
seekPending = true;
|
||||
indexIsCurrent = false;
|
||||
term.copyBytes(target);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermState termState() throws IOException {
|
||||
//System.out.println("BTR.termState this=" + this);
|
||||
decodeMetaData();
|
||||
TermState ts = state.clone();
|
||||
//System.out.println(" return ts=" + ts);
|
||||
return ts;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seekExact(long ord) throws IOException {
|
||||
//System.out.println("BTR.seek by ord ord=" + ord);
|
||||
if (indexEnum == null) {
|
||||
throw new IllegalStateException("terms index was not loaded");
|
||||
}
|
||||
|
||||
assert ord < numTerms;
|
||||
|
||||
// TODO: if ord is in same terms block and
|
||||
// after current ord, we should avoid this seek just
|
||||
// like we do in the seek(BytesRef) case
|
||||
in.seek(indexEnum.seek(ord));
|
||||
boolean result = nextBlock();
|
||||
|
||||
// Block must exist since ord < numTerms:
|
||||
assert result;
|
||||
|
||||
indexIsCurrent = true;
|
||||
didIndexNext = false;
|
||||
seekPending = false;
|
||||
|
||||
state.ord = indexEnum.ord()-1;
|
||||
assert state.ord >= -1: "ord=" + state.ord;
|
||||
term.copyBytes(indexEnum.term());
|
||||
|
||||
// Now, scan:
|
||||
int left = (int) (ord - state.ord);
|
||||
while(left > 0) {
|
||||
final BytesRef term = _next();
|
||||
assert term != null;
|
||||
left--;
|
||||
assert indexIsCurrent;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ord() {
|
||||
if (!doOrd) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
return state.ord;
|
||||
}
|
||||
|
||||
/* 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. */
|
||||
private boolean nextBlock() throws IOException {
|
||||
|
||||
// TODO: we still lazy-decode the byte[] for each
|
||||
// term (the suffix), but, if we decoded
|
||||
// all N terms up front then seeking could do a fast
|
||||
// bsearch w/in the block...
|
||||
|
||||
//System.out.println("BTR.nextBlock() fp=" + in.getFilePointer() + " this=" + this);
|
||||
state.blockFilePointer = in.getFilePointer();
|
||||
blockTermCount = in.readVInt();
|
||||
//System.out.println(" blockTermCount=" + blockTermCount);
|
||||
if (blockTermCount == 0) {
|
||||
return false;
|
||||
}
|
||||
termBlockPrefix = in.readVInt();
|
||||
|
||||
// term suffixes:
|
||||
int len = in.readVInt();
|
||||
if (termSuffixes.length < len) {
|
||||
termSuffixes = new byte[ArrayUtil.oversize(len, 1)];
|
||||
}
|
||||
//System.out.println(" termSuffixes len=" + len);
|
||||
in.readBytes(termSuffixes, 0, len);
|
||||
termSuffixesReader.reset(termSuffixes, 0, len);
|
||||
|
||||
// docFreq, totalTermFreq
|
||||
len = in.readVInt();
|
||||
if (docFreqBytes.length < len) {
|
||||
docFreqBytes = new byte[ArrayUtil.oversize(len, 1)];
|
||||
}
|
||||
//System.out.println(" freq bytes len=" + len);
|
||||
in.readBytes(docFreqBytes, 0, len);
|
||||
freqReader.reset(docFreqBytes, 0, len);
|
||||
|
||||
// metadata
|
||||
len = in.readVInt();
|
||||
if (bytes == null) {
|
||||
bytes = new byte[ArrayUtil.oversize(len, 1)];
|
||||
bytesReader = new ByteArrayDataInput();
|
||||
} else if (bytes.length < len) {
|
||||
bytes = new byte[ArrayUtil.oversize(len, 1)];
|
||||
}
|
||||
in.readBytes(bytes, 0, len);
|
||||
bytesReader.reset(bytes, 0, len);
|
||||
|
||||
metaDataUpto = 0;
|
||||
state.termBlockOrd = 0;
|
||||
|
||||
indexIsCurrent = false;
|
||||
//System.out.println(" indexIsCurrent=" + indexIsCurrent);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private void decodeMetaData() throws IOException {
|
||||
//System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termBlockOrd + " state=" + state);
|
||||
if (!seekPending) {
|
||||
// TODO: cutover to random-access API
|
||||
// here.... really stupid that we have to decode N
|
||||
// wasted term metadata just to get to the N+1th
|
||||
// that we really need...
|
||||
|
||||
// lazily catch up on metadata decode:
|
||||
final int limit = state.termBlockOrd;
|
||||
// We must set/incr state.termCount because
|
||||
// postings impl can look at this
|
||||
state.termBlockOrd = metaDataUpto;
|
||||
if (metaDataUpto == 0) {
|
||||
Arrays.fill(longs, 0);
|
||||
}
|
||||
// TODO: better API would be "jump straight to term=N"???
|
||||
while (metaDataUpto < limit) {
|
||||
//System.out.println(" decode mdUpto=" + metaDataUpto);
|
||||
// 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:
|
||||
|
||||
// docFreq, totalTermFreq
|
||||
state.docFreq = freqReader.readVInt();
|
||||
//System.out.println(" dF=" + state.docFreq);
|
||||
if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
|
||||
state.totalTermFreq = state.docFreq + freqReader.readVLong();
|
||||
//System.out.println(" totTF=" + state.totalTermFreq);
|
||||
}
|
||||
// metadata
|
||||
for (int i = 0; i < longs.length; i++) {
|
||||
longs[i] += bytesReader.readVLong();
|
||||
}
|
||||
postingsReader.decodeTerm(longs, bytesReader, fieldInfo, state);
|
||||
metaDataUpto++;
|
||||
state.termBlockOrd++;
|
||||
}
|
||||
} else {
|
||||
//System.out.println(" skip! seekPending");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,364 @@
|
|||
package org.apache.lucene.codecs.temp;
|
||||
|
||||
/*
|
||||
* 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.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.PostingsConsumer;
|
||||
import org.apache.lucene.codecs.TempPostingsWriterBase;
|
||||
import org.apache.lucene.codecs.TermStats;
|
||||
import org.apache.lucene.codecs.TermsConsumer;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.codecs.blockterms.*;
|
||||
|
||||
// TODO: currently we encode all terms between two indexed
|
||||
// terms as a block; but, we could decouple the two, ie
|
||||
// allow several blocks in between two indexed terms
|
||||
|
||||
/**
|
||||
* Writes terms dict, block-encoding (column stride) each
|
||||
* term's metadata for each set of terms between two
|
||||
* index terms.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public class TempBlockTermsWriter extends FieldsConsumer {
|
||||
|
||||
final static String CODEC_NAME = "BLOCK_TERMS_DICT";
|
||||
|
||||
// Initial format
|
||||
public static final int VERSION_START = 0;
|
||||
public static final int VERSION_APPEND_ONLY = 1;
|
||||
public static final int VERSION_CURRENT = VERSION_APPEND_ONLY;
|
||||
|
||||
/** Extension of terms file */
|
||||
static final String TERMS_EXTENSION = "tib";
|
||||
|
||||
protected final IndexOutput out;
|
||||
final TempPostingsWriterBase postingsWriter;
|
||||
final FieldInfos fieldInfos;
|
||||
FieldInfo currentField;
|
||||
private final TermsIndexWriterBase termsIndexWriter;
|
||||
|
||||
private static class FieldMetaData {
|
||||
public final FieldInfo fieldInfo;
|
||||
public final long numTerms;
|
||||
public final long termsStartPointer;
|
||||
public final long sumTotalTermFreq;
|
||||
public final long sumDocFreq;
|
||||
public final int docCount;
|
||||
public final int longsSize;
|
||||
|
||||
public FieldMetaData(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize) {
|
||||
assert numTerms > 0;
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.termsStartPointer = termsStartPointer;
|
||||
this.numTerms = numTerms;
|
||||
this.sumTotalTermFreq = sumTotalTermFreq;
|
||||
this.sumDocFreq = sumDocFreq;
|
||||
this.docCount = docCount;
|
||||
this.longsSize = longsSize;
|
||||
}
|
||||
}
|
||||
|
||||
private final List<FieldMetaData> fields = new ArrayList<FieldMetaData>();
|
||||
|
||||
// private final String segment;
|
||||
|
||||
public TempBlockTermsWriter(TermsIndexWriterBase termsIndexWriter,
|
||||
SegmentWriteState state, TempPostingsWriterBase postingsWriter)
|
||||
throws IOException {
|
||||
final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
|
||||
this.termsIndexWriter = termsIndexWriter;
|
||||
out = state.directory.createOutput(termsFileName, state.context);
|
||||
boolean success = false;
|
||||
try {
|
||||
fieldInfos = state.fieldInfos;
|
||||
writeHeader(out);
|
||||
currentField = null;
|
||||
this.postingsWriter = postingsWriter;
|
||||
// segment = state.segmentName;
|
||||
|
||||
//System.out.println("BTW.init seg=" + state.segmentName);
|
||||
|
||||
postingsWriter.start(out); // have consumer write its format/header
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void writeHeader(IndexOutput out) throws IOException {
|
||||
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsConsumer addField(FieldInfo field) throws IOException {
|
||||
//System.out.println("\nBTW.addField seg=" + segment + " field=" + field.name);
|
||||
assert currentField == null || currentField.name.compareTo(field.name) < 0;
|
||||
currentField = field;
|
||||
TermsIndexWriterBase.FieldWriter fieldIndexWriter = termsIndexWriter.addField(field, out.getFilePointer());
|
||||
return new TermsWriter(fieldIndexWriter, field, postingsWriter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
|
||||
try {
|
||||
|
||||
final long dirStart = out.getFilePointer();
|
||||
|
||||
out.writeVInt(fields.size());
|
||||
for(FieldMetaData field : fields) {
|
||||
out.writeVInt(field.fieldInfo.number);
|
||||
out.writeVLong(field.numTerms);
|
||||
out.writeVLong(field.termsStartPointer);
|
||||
if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
|
||||
out.writeVLong(field.sumTotalTermFreq);
|
||||
}
|
||||
out.writeVLong(field.sumDocFreq);
|
||||
out.writeVInt(field.docCount);
|
||||
out.writeVInt(field.longsSize);
|
||||
}
|
||||
writeTrailer(dirStart);
|
||||
} finally {
|
||||
IOUtils.close(out, postingsWriter, termsIndexWriter);
|
||||
}
|
||||
}
|
||||
|
||||
private void writeTrailer(long dirStart) throws IOException {
|
||||
out.writeLong(dirStart);
|
||||
}
|
||||
|
||||
private static class TermEntry {
|
||||
public final BytesRef term = new BytesRef();
|
||||
public TermStats stats;
|
||||
public long[] longs;
|
||||
public byte[] bytes;
|
||||
}
|
||||
|
||||
class TermsWriter extends TermsConsumer {
|
||||
private final FieldInfo fieldInfo;
|
||||
private final TempPostingsWriterBase postingsWriter;
|
||||
private final long termsStartPointer;
|
||||
private long numTerms;
|
||||
private final TermsIndexWriterBase.FieldWriter fieldIndexWriter;
|
||||
long sumTotalTermFreq;
|
||||
long sumDocFreq;
|
||||
int docCount;
|
||||
int longsSize;
|
||||
|
||||
private TermEntry[] pendingTerms;
|
||||
|
||||
private int pendingCount;
|
||||
|
||||
TermsWriter(
|
||||
TermsIndexWriterBase.FieldWriter fieldIndexWriter,
|
||||
FieldInfo fieldInfo,
|
||||
TempPostingsWriterBase postingsWriter)
|
||||
{
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.fieldIndexWriter = fieldIndexWriter;
|
||||
pendingTerms = new TermEntry[32];
|
||||
for(int i=0;i<pendingTerms.length;i++) {
|
||||
pendingTerms[i] = new TermEntry();
|
||||
}
|
||||
termsStartPointer = out.getFilePointer();
|
||||
this.postingsWriter = postingsWriter;
|
||||
this.longsSize = postingsWriter.setField(fieldInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsConsumer startTerm(BytesRef text) throws IOException {
|
||||
//System.out.println("BTW: startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
|
||||
postingsWriter.startTerm();
|
||||
return postingsWriter;
|
||||
}
|
||||
|
||||
private final BytesRef lastPrevTerm = new BytesRef();
|
||||
|
||||
@Override
|
||||
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
|
||||
|
||||
assert stats.docFreq > 0;
|
||||
//System.out.println("BTW: finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
|
||||
|
||||
final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, stats);
|
||||
|
||||
if (isIndexTerm) {
|
||||
if (pendingCount > 0) {
|
||||
// Instead of writing each term, live, we gather terms
|
||||
// in RAM in a pending buffer, and then write the
|
||||
// entire block in between index terms:
|
||||
flushBlock();
|
||||
}
|
||||
fieldIndexWriter.add(text, stats, out.getFilePointer());
|
||||
//System.out.println(" index term!");
|
||||
}
|
||||
|
||||
if (pendingTerms.length == pendingCount) {
|
||||
final TermEntry[] newArray = new TermEntry[ArrayUtil.oversize(pendingCount+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
|
||||
System.arraycopy(pendingTerms, 0, newArray, 0, pendingCount);
|
||||
for(int i=pendingCount;i<newArray.length;i++) {
|
||||
newArray[i] = new TermEntry();
|
||||
}
|
||||
pendingTerms = newArray;
|
||||
}
|
||||
final TermEntry te = pendingTerms[pendingCount];
|
||||
te.term.copyBytes(text);
|
||||
te.stats = stats;
|
||||
te.longs = new long[longsSize];
|
||||
postingsWriter.finishTerm(te.longs, bytesWriter, stats);
|
||||
te.bytes = new byte[(int) bytesWriter.getFilePointer()];
|
||||
bytesWriter.writeTo(te.bytes, 0);
|
||||
bytesWriter.reset();
|
||||
|
||||
pendingCount++;
|
||||
numTerms++;
|
||||
}
|
||||
|
||||
// Finishes all terms in this field
|
||||
@Override
|
||||
public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
|
||||
if (pendingCount > 0) {
|
||||
flushBlock();
|
||||
}
|
||||
// EOF marker:
|
||||
out.writeVInt(0);
|
||||
|
||||
this.sumTotalTermFreq = sumTotalTermFreq;
|
||||
this.sumDocFreq = sumDocFreq;
|
||||
this.docCount = docCount;
|
||||
fieldIndexWriter.finish(out.getFilePointer());
|
||||
if (numTerms > 0) {
|
||||
fields.add(new FieldMetaData(fieldInfo,
|
||||
numTerms,
|
||||
termsStartPointer,
|
||||
sumTotalTermFreq,
|
||||
sumDocFreq,
|
||||
docCount,
|
||||
longsSize));
|
||||
}
|
||||
}
|
||||
|
||||
private int sharedPrefix(BytesRef term1, BytesRef term2) {
|
||||
assert term1.offset == 0;
|
||||
assert term2.offset == 0;
|
||||
int pos1 = 0;
|
||||
int pos1End = pos1 + Math.min(term1.length, term2.length);
|
||||
int pos2 = 0;
|
||||
while(pos1 < pos1End) {
|
||||
if (term1.bytes[pos1] != term2.bytes[pos2]) {
|
||||
return pos1;
|
||||
}
|
||||
pos1++;
|
||||
pos2++;
|
||||
}
|
||||
return pos1;
|
||||
}
|
||||
|
||||
private final RAMOutputStream bytesWriter = new RAMOutputStream();
|
||||
|
||||
private void flushBlock() throws IOException {
|
||||
//System.out.println("BTW.flushBlock seg=" + segment + " pendingCount=" + pendingCount + " fp=" + out.getFilePointer());
|
||||
|
||||
// First pass: compute common prefix for all terms
|
||||
// in the block, against term before first term in
|
||||
// this block:
|
||||
int commonPrefix = sharedPrefix(lastPrevTerm, pendingTerms[0].term);
|
||||
for(int termCount=1;termCount<pendingCount;termCount++) {
|
||||
commonPrefix = Math.min(commonPrefix,
|
||||
sharedPrefix(lastPrevTerm,
|
||||
pendingTerms[termCount].term));
|
||||
}
|
||||
|
||||
out.writeVInt(pendingCount);
|
||||
out.writeVInt(commonPrefix);
|
||||
|
||||
// 2nd pass: write suffixes, as separate byte[] blob
|
||||
for(int termCount=0;termCount<pendingCount;termCount++) {
|
||||
final int suffix = pendingTerms[termCount].term.length - commonPrefix;
|
||||
// TODO: cutover to better intblock codec, instead
|
||||
// of interleaving here:
|
||||
bytesWriter.writeVInt(suffix);
|
||||
bytesWriter.writeBytes(pendingTerms[termCount].term.bytes, commonPrefix, suffix);
|
||||
}
|
||||
out.writeVInt((int) bytesWriter.getFilePointer());
|
||||
bytesWriter.writeTo(out);
|
||||
bytesWriter.reset();
|
||||
|
||||
// 3rd pass: write the freqs as byte[] blob
|
||||
// TODO: cutover to better intblock codec. simple64?
|
||||
// write prefix, suffix first:
|
||||
for(int termCount=0;termCount<pendingCount;termCount++) {
|
||||
final TermStats stats = pendingTerms[termCount].stats;
|
||||
assert stats != null;
|
||||
bytesWriter.writeVInt(stats.docFreq);
|
||||
if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
|
||||
bytesWriter.writeVLong(stats.totalTermFreq-stats.docFreq);
|
||||
}
|
||||
}
|
||||
out.writeVInt((int) bytesWriter.getFilePointer());
|
||||
bytesWriter.writeTo(out);
|
||||
bytesWriter.reset();
|
||||
|
||||
// 4th pass: write the metadata
|
||||
long[] lastLongs = new long[longsSize];
|
||||
Arrays.fill(lastLongs, 0);
|
||||
for(int termCount=0;termCount<pendingCount;termCount++) {
|
||||
final long[] longs = pendingTerms[termCount].longs;
|
||||
final byte[] bytes = pendingTerms[termCount].bytes;
|
||||
for (int i = 0; i < longsSize; i++) {
|
||||
bytesWriter.writeVLong(longs[i] - lastLongs[i]);
|
||||
}
|
||||
lastLongs = longs;
|
||||
bytesWriter.writeBytes(bytes, 0, bytes.length);
|
||||
}
|
||||
out.writeVInt((int) bytesWriter.getFilePointer());
|
||||
bytesWriter.writeTo(out);
|
||||
bytesWriter.reset();
|
||||
|
||||
lastPrevTerm.copyBytes(pendingTerms[pendingCount-1].term);
|
||||
pendingCount = 0;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -18,3 +18,4 @@ org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat
|
|||
org.apache.lucene.codecs.memory.MemoryPostingsFormat
|
||||
org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
|
||||
org.apache.lucene.codecs.memory.DirectPostingsFormat
|
||||
org.apache.lucene.codecs.temp.TempBlockPostingsFormat
|
||||
|
|
|
@ -56,7 +56,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
*
|
||||
* <li>
|
||||
* <b>Block structure</b>:
|
||||
* <p>When the postings are long enough, TempBlockPostingsFormat will try to encode most integer data
|
||||
* <p>When the postings are long enough, TempBlockTreePostingsFormat will try to encode most integer data
|
||||
* as a packed block.</p>
|
||||
* <p>Take a term with 259 documents as an example, the first 256 document ids are encoded as two packed
|
||||
* blocks, while the remaining 3 are encoded as one VInt block. </p>
|
||||
|
@ -116,7 +116,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* field along with per-term statistics (such as docfreq)
|
||||
* and pointers to the frequencies, positions, payload and
|
||||
* skip data in the .doc, .pos, and .pay files.
|
||||
* See {@link TempBlockTermsWriter} for more details on the format.
|
||||
* See {@link TempBlockTreeTermsWriter} for more details on the format.
|
||||
* </p>
|
||||
*
|
||||
* <p>NOTE: The term dictionary can plug into different postings implementations:
|
||||
|
@ -159,7 +159,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* <li>SkipFPDelta determines the position of this term's SkipData within the .doc
|
||||
* file. In particular, it is the length of the TermFreq data.
|
||||
* SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
|
||||
* (i.e. 8 in TempBlockPostingsFormat).</li>
|
||||
* (i.e. 8 in TempBlockTreePostingsFormat).</li>
|
||||
* <li>SingletonDocID is an optimization when a term only appears in one document. In this case, instead
|
||||
* of writing a file pointer to the .doc file (DocFPDelta), and then a VIntBlock at that location, the
|
||||
* single document ID is written to the term dictionary.</li>
|
||||
|
@ -172,7 +172,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* <dd>
|
||||
* <b>Term Index</b>
|
||||
* <p>The .tip file contains an index into the term dictionary, so that it can be
|
||||
* accessed randomly. See {@link TempBlockTermsWriter} for more details on the format.</p>
|
||||
* accessed randomly. See {@link TempBlockTreeTermsWriter} for more details on the format.</p>
|
||||
* </dd>
|
||||
* </dl>
|
||||
*
|
||||
|
@ -239,7 +239,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* We use this trick since the definition of skip entry is a little different from base interface.
|
||||
* In {@link MultiLevelSkipListWriter}, skip data is assumed to be saved for
|
||||
* skipInterval<sup>th</sup>, 2*skipInterval<sup>th</sup> ... posting in the list. However,
|
||||
* in TempBlockPostingsFormat, the skip data is saved for skipInterval+1<sup>th</sup>,
|
||||
* in TempBlockTreePostingsFormat, the skip data is saved for skipInterval+1<sup>th</sup>,
|
||||
* 2*skipInterval+1<sup>th</sup> ... posting (skipInterval==PackedBlockSize in this case).
|
||||
* When DocFreq is multiple of PackedBlockSize, MultiLevelSkipListWriter will expect one
|
||||
* more skip data than TempSkipWriter. </li>
|
||||
|
@ -352,7 +352,7 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public final class TempBlockPostingsFormat extends PostingsFormat {
|
||||
public final class TempBlockTreePostingsFormat extends PostingsFormat {
|
||||
/**
|
||||
* Filename extension for document number, frequencies, and skip data.
|
||||
* See chapter: <a href="#Frequencies">Frequencies and Skip Data</a>
|
||||
|
@ -381,18 +381,18 @@ public final class TempBlockPostingsFormat extends PostingsFormat {
|
|||
// NOTE: must be multiple of 64 because of PackedInts long-aligned encoding/decoding
|
||||
public final static int BLOCK_SIZE = 128;
|
||||
|
||||
/** Creates {@code TempBlockPostingsFormat} with default
|
||||
/** Creates {@code TempBlockTreePostingsFormat} with default
|
||||
* settings. */
|
||||
public TempBlockPostingsFormat() {
|
||||
this(TempBlockTermsWriter.DEFAULT_MIN_BLOCK_SIZE, TempBlockTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
|
||||
public TempBlockTreePostingsFormat() {
|
||||
this(TempBlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, TempBlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
|
||||
}
|
||||
|
||||
/** Creates {@code TempBlockPostingsFormat} with custom
|
||||
/** Creates {@code TempBlockTreePostingsFormat} with custom
|
||||
* values for {@code minBlockSize} and {@code
|
||||
* maxBlockSize} passed to block terms dictionary.
|
||||
* @see TempBlockTermsWriter#TempBlockTermsWriter(SegmentWriteState,TempPostingsWriterBase,int,int) */
|
||||
public TempBlockPostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
|
||||
super("TempBlock");
|
||||
* @see TempBlockTreeTermsWriter#TempBlockTreeTermsWriter(SegmentWriteState,TempPostingsWriterBase,int,int) */
|
||||
public TempBlockTreePostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
|
||||
super("TempBlockTree");
|
||||
this.minTermBlockSize = minTermBlockSize;
|
||||
assert minTermBlockSize > 1;
|
||||
this.maxTermBlockSize = maxTermBlockSize;
|
||||
|
@ -410,7 +410,7 @@ public final class TempBlockPostingsFormat extends PostingsFormat {
|
|||
|
||||
boolean success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new TempBlockTermsWriter(state,
|
||||
FieldsConsumer ret = new TempBlockTreeTermsWriter(state,
|
||||
postingsWriter,
|
||||
minTermBlockSize,
|
||||
maxTermBlockSize);
|
||||
|
@ -432,7 +432,7 @@ public final class TempBlockPostingsFormat extends PostingsFormat {
|
|||
state.segmentSuffix);
|
||||
boolean success = false;
|
||||
try {
|
||||
FieldsProducer ret = new TempBlockTermsReader(state.directory,
|
||||
FieldsProducer ret = new TempBlockTreeTermsReader(state.directory,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo,
|
||||
postingsReader,
|
|
@ -72,9 +72,9 @@ import org.apache.lucene.codecs.PostingsBaseFormat; // javadoc
|
|||
* does not support a pluggable terms index
|
||||
* implementation).
|
||||
*
|
||||
* <p><b>NOTE</b>: this terms dictionary does not support
|
||||
* index divisor when opening an IndexReader. Instead, you
|
||||
* can change the min/maxItemsPerBlock during indexing.</p>
|
||||
* <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
|
||||
|
@ -86,17 +86,17 @@ import org.apache.lucene.codecs.PostingsBaseFormat; // javadoc
|
|||
* option to see summary statistics on the blocks in the
|
||||
* dictionary.
|
||||
*
|
||||
* See {@link TempBlockTermsWriter}.
|
||||
* See {@link TempBlockTreeTermsWriter}.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public class TempBlockTermsReader extends FieldsProducer {
|
||||
public class TempBlockTreeTermsReader extends FieldsProducer {
|
||||
|
||||
// Open input to the main terms dict file (_X.tib)
|
||||
private final IndexInput in;
|
||||
|
||||
//private static final boolean DEBUG = TempBlockTermsWriter.DEBUG;
|
||||
//private static final boolean DEBUG = TempBlockTreeTermsWriter.DEBUG;
|
||||
|
||||
// Reads the terms dict entries, to gather state to
|
||||
// produce DocsEnum on demand
|
||||
|
@ -115,7 +115,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
private final int version;
|
||||
|
||||
/** Sole constructor. */
|
||||
public TempBlockTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
|
||||
public TempBlockTreeTermsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo info,
|
||||
TempPostingsReaderBase postingsReader, IOContext ioContext,
|
||||
String segmentSuffix)
|
||||
throws IOException {
|
||||
|
@ -123,7 +123,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
this.postingsReader = postingsReader;
|
||||
|
||||
this.segment = info.name;
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, TempBlockTermsWriter.TERMS_EXTENSION),
|
||||
in = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, TempBlockTreeTermsWriter.TERMS_EXTENSION),
|
||||
ioContext);
|
||||
|
||||
boolean success = false;
|
||||
|
@ -131,7 +131,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
|
||||
try {
|
||||
version = readHeader(in);
|
||||
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, TempBlockTermsWriter.TERMS_INDEX_EXTENSION),
|
||||
indexIn = dir.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, TempBlockTreeTermsWriter.TERMS_INDEX_EXTENSION),
|
||||
ioContext);
|
||||
int indexVersion = readIndexHeader(indexIn);
|
||||
if (indexVersion != version) {
|
||||
|
@ -192,10 +192,10 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
|
||||
/** Reads terms file header. */
|
||||
private int readHeader(IndexInput input) throws IOException {
|
||||
int version = CodecUtil.checkHeader(input, TempBlockTermsWriter.TERMS_CODEC_NAME,
|
||||
TempBlockTermsWriter.TERMS_VERSION_START,
|
||||
TempBlockTermsWriter.TERMS_VERSION_CURRENT);
|
||||
if (version < TempBlockTermsWriter.TERMS_VERSION_APPEND_ONLY) {
|
||||
int version = CodecUtil.checkHeader(input, TempBlockTreeTermsWriter.TERMS_CODEC_NAME,
|
||||
TempBlockTreeTermsWriter.TERMS_VERSION_START,
|
||||
TempBlockTreeTermsWriter.TERMS_VERSION_CURRENT);
|
||||
if (version < TempBlockTreeTermsWriter.TERMS_VERSION_APPEND_ONLY) {
|
||||
dirOffset = input.readLong();
|
||||
}
|
||||
return version;
|
||||
|
@ -203,10 +203,10 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
|
||||
/** Reads index file header. */
|
||||
private int readIndexHeader(IndexInput input) throws IOException {
|
||||
int version = CodecUtil.checkHeader(input, TempBlockTermsWriter.TERMS_INDEX_CODEC_NAME,
|
||||
TempBlockTermsWriter.TERMS_INDEX_VERSION_START,
|
||||
TempBlockTermsWriter.TERMS_INDEX_VERSION_CURRENT);
|
||||
if (version < TempBlockTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
|
||||
int version = CodecUtil.checkHeader(input, TempBlockTreeTermsWriter.TERMS_INDEX_CODEC_NAME,
|
||||
TempBlockTreeTermsWriter.TERMS_INDEX_VERSION_START,
|
||||
TempBlockTreeTermsWriter.TERMS_INDEX_VERSION_CURRENT);
|
||||
if (version < TempBlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
|
||||
indexDirOffset = input.readLong();
|
||||
}
|
||||
return version;
|
||||
|
@ -215,7 +215,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
/** Seek {@code input} to the directory offset. */
|
||||
private void seekDir(IndexInput input, long dirOffset)
|
||||
throws IOException {
|
||||
if (version >= TempBlockTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
|
||||
if (version >= TempBlockTreeTermsWriter.TERMS_INDEX_VERSION_APPEND_ONLY) {
|
||||
input.seek(input.length() - 8);
|
||||
dirOffset = input.readLong();
|
||||
}
|
||||
|
@ -462,7 +462,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount, long indexStartFP, int longsSize, IndexInput indexIn) throws IOException {
|
||||
assert numTerms > 0;
|
||||
this.fieldInfo = fieldInfo;
|
||||
//DEBUG = TempBlockTermsReader.DEBUG && fieldInfo.name.equals("id");
|
||||
//DEBUG = TempBlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
|
||||
this.numTerms = numTerms;
|
||||
this.sumTotalTermFreq = sumTotalTermFreq;
|
||||
this.sumDocFreq = sumDocFreq;
|
||||
|
@ -474,7 +474,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
// System.out.println("BTTR: seg=" + segment + " field=" + fieldInfo.name + " rootBlockCode=" + rootCode + " divisor=" + indexDivisor);
|
||||
// }
|
||||
|
||||
rootBlockFP = (new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong() >>> TempBlockTermsWriter.OUTPUT_FLAGS_NUM_BITS;
|
||||
rootBlockFP = (new ByteArrayDataInput(rootCode.bytes, rootCode.offset, rootCode.length)).readVLong() >>> TempBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
|
||||
|
||||
if (indexIn != null) {
|
||||
final IndexInput clone = indexIn.clone();
|
||||
|
@ -684,7 +684,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
// Skip first long -- has redundant fp, hasTerms
|
||||
// flag, isFloor flag
|
||||
final long code = floorDataReader.readVLong();
|
||||
if ((code & TempBlockTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) {
|
||||
if ((code & TempBlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0) {
|
||||
numFollowFloorBlocks = floorDataReader.readVInt();
|
||||
nextFloorLabel = floorDataReader.readByte() & 0xff;
|
||||
// if (DEBUG) System.out.println(" numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + nextFloorLabel);
|
||||
|
@ -844,7 +844,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
// }
|
||||
runAutomaton = compiled.runAutomaton;
|
||||
compiledAutomaton = compiled;
|
||||
in = TempBlockTermsReader.this.in.clone();
|
||||
in = TempBlockTreeTermsReader.this.in.clone();
|
||||
stack = new Frame[5];
|
||||
for(int idx=0;idx<stack.length;idx++) {
|
||||
stack[idx] = new Frame(idx);
|
||||
|
@ -1334,7 +1334,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
// Not private to avoid synthetic access$NNN methods
|
||||
void initIndexInput() {
|
||||
if (this.in == null) {
|
||||
this.in = TempBlockTermsReader.this.in.clone();
|
||||
this.in = TempBlockTreeTermsReader.this.in.clone();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1464,11 +1464,11 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
Frame pushFrame(FST.Arc<BytesRef> arc, BytesRef frameData, int length) throws IOException {
|
||||
scratchReader.reset(frameData.bytes, frameData.offset, frameData.length);
|
||||
final long code = scratchReader.readVLong();
|
||||
final long fpSeek = code >>> TempBlockTermsWriter.OUTPUT_FLAGS_NUM_BITS;
|
||||
final long fpSeek = code >>> TempBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS;
|
||||
final Frame f = getFrame(1+currentFrame.ord);
|
||||
f.hasTerms = (code & TempBlockTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0;
|
||||
f.hasTerms = (code & TempBlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS) != 0;
|
||||
f.hasTermsOrig = f.hasTerms;
|
||||
f.isFloor = (code & TempBlockTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0;
|
||||
f.isFloor = (code & TempBlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR) != 0;
|
||||
if (f.isFloor) {
|
||||
f.setFloorData(scratchReader, frameData);
|
||||
}
|
||||
|
@ -2048,9 +2048,9 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
assert f != null;
|
||||
final BytesRef prefix = new BytesRef(term.bytes, 0, f.prefix);
|
||||
if (f.nextEnt == -1) {
|
||||
out.println(" frame " + (isSeekFrame ? "(seek)" : "(next)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<TempBlockTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? TempBlockTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? TempBlockTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
|
||||
out.println(" frame " + (isSeekFrame ? "(seek)" : "(next)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<TempBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? TempBlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? TempBlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
|
||||
} else {
|
||||
out.println(" frame " + (isSeekFrame ? "(seek, loaded)" : "(next, loaded)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + " nextEnt=" + f.nextEnt + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<TempBlockTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? TempBlockTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? TempBlockTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " lastSubFP=" + f.lastSubFP + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
|
||||
out.println(" frame " + (isSeekFrame ? "(seek, loaded)" : "(next, loaded)") + " ord=" + ord + " fp=" + f.fp + (f.isFloor ? (" (fpOrig=" + f.fpOrig + ")") : "") + " prefixLen=" + f.prefix + " prefix=" + prefix + " nextEnt=" + f.nextEnt + (f.nextEnt == -1 ? "" : (" (of " + f.entCount + ")")) + " hasTerms=" + f.hasTerms + " isFloor=" + f.isFloor + " code=" + ((f.fp<<TempBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) + (f.hasTerms ? TempBlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) + (f.isFloor ? TempBlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0)) + " lastSubFP=" + f.lastSubFP + " isLastInFloor=" + f.isLastInFloor + " mdUpto=" + f.metaDataUpto + " tbOrd=" + f.getTermBlockOrd());
|
||||
}
|
||||
if (index != null) {
|
||||
assert !isSeekFrame || f.arc != null: "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc;
|
||||
|
@ -2065,7 +2065,7 @@ public class TempBlockTermsReader extends FieldsProducer {
|
|||
} else if (isSeekFrame && !f.isFloor) {
|
||||
final ByteArrayDataInput reader = new ByteArrayDataInput(output.bytes, output.offset, output.length);
|
||||
final long codeOrig = reader.readVLong();
|
||||
final long code = (f.fp << TempBlockTermsWriter.OUTPUT_FLAGS_NUM_BITS) | (f.hasTerms ? TempBlockTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) | (f.isFloor ? TempBlockTermsWriter.OUTPUT_FLAG_IS_FLOOR:0);
|
||||
final long code = (f.fp << TempBlockTreeTermsWriter.OUTPUT_FLAGS_NUM_BITS) | (f.hasTerms ? TempBlockTreeTermsWriter.OUTPUT_FLAG_HAS_TERMS:0) | (f.isFloor ? TempBlockTreeTermsWriter.OUTPUT_FLAG_IS_FLOOR:0);
|
||||
if (codeOrig != code) {
|
||||
out.println(" broken seek state: output code=" + codeOrig + " doesn't match frame code=" + code);
|
||||
throw new RuntimeException("seek state is broken");
|
|
@ -178,20 +178,20 @@ import org.apache.lucene.codecs.CodecUtil;
|
|||
* sub-block, and its file pointer.
|
||||
* </ul>
|
||||
*
|
||||
* @see TempBlockTermsReader
|
||||
* @see TempBlockTreeTermsReader
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public class TempBlockTermsWriter extends FieldsConsumer {
|
||||
public class TempBlockTreeTermsWriter extends FieldsConsumer {
|
||||
|
||||
/** Suggested default value for the {@code
|
||||
* minItemsInBlock} parameter to {@link
|
||||
* #TempBlockTermsWriter(SegmentWriteState,TempPostingsWriterBase,int,int)}. */
|
||||
* #TempBlockTreeTermsWriter(SegmentWriteState,TempPostingsWriterBase,int,int)}. */
|
||||
public final static int DEFAULT_MIN_BLOCK_SIZE = 25;
|
||||
|
||||
/** Suggested default value for the {@code
|
||||
* maxItemsInBlock} parameter to {@link
|
||||
* #TempBlockTermsWriter(SegmentWriteState,TempPostingsWriterBase,int,int)}. */
|
||||
* #TempBlockTreeTermsWriter(SegmentWriteState,TempPostingsWriterBase,int,int)}. */
|
||||
public final static int DEFAULT_MAX_BLOCK_SIZE = 48;
|
||||
|
||||
//public final static boolean DEBUG = false;
|
||||
|
@ -268,7 +268,7 @@ public class TempBlockTermsWriter extends FieldsConsumer {
|
|||
* sub-blocks) per block will aim to be between
|
||||
* minItemsPerBlock and maxItemsPerBlock, though in some
|
||||
* cases the blocks may be smaller than the min. */
|
||||
public TempBlockTermsWriter(
|
||||
public TempBlockTreeTermsWriter(
|
||||
SegmentWriteState state,
|
||||
TempPostingsWriterBase postingsWriter,
|
||||
int minItemsInBlock,
|
|
@ -71,7 +71,7 @@ public final class TempPostingsReader extends TempPostingsReaderBase {
|
|||
IndexInput posIn = null;
|
||||
IndexInput payIn = null;
|
||||
try {
|
||||
docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, TempBlockPostingsFormat.DOC_EXTENSION),
|
||||
docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, TempBlockTreePostingsFormat.DOC_EXTENSION),
|
||||
ioContext);
|
||||
CodecUtil.checkHeader(docIn,
|
||||
TempPostingsWriter.DOC_CODEC,
|
||||
|
@ -80,7 +80,7 @@ public final class TempPostingsReader extends TempPostingsReaderBase {
|
|||
forUtil = new ForUtil(docIn);
|
||||
|
||||
if (fieldInfos.hasProx()) {
|
||||
posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, TempBlockPostingsFormat.POS_EXTENSION),
|
||||
posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, TempBlockTreePostingsFormat.POS_EXTENSION),
|
||||
ioContext);
|
||||
CodecUtil.checkHeader(posIn,
|
||||
TempPostingsWriter.POS_CODEC,
|
||||
|
@ -88,7 +88,7 @@ public final class TempPostingsReader extends TempPostingsReaderBase {
|
|||
TempPostingsWriter.VERSION_CURRENT);
|
||||
|
||||
if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
|
||||
payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, TempBlockPostingsFormat.PAY_EXTENSION),
|
||||
payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, TempBlockTreePostingsFormat.PAY_EXTENSION),
|
||||
ioContext);
|
||||
CodecUtil.checkHeader(payIn,
|
||||
TempPostingsWriter.PAY_CODEC,
|
||||
|
|
|
@ -119,7 +119,7 @@ public final class TempPostingsWriter extends TempPostingsWriterBase {
|
|||
public TempPostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
|
||||
super();
|
||||
|
||||
docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempBlockPostingsFormat.DOC_EXTENSION),
|
||||
docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempBlockTreePostingsFormat.DOC_EXTENSION),
|
||||
state.context);
|
||||
IndexOutput posOut = null;
|
||||
IndexOutput payOut = null;
|
||||
|
@ -129,7 +129,7 @@ public final class TempPostingsWriter extends TempPostingsWriterBase {
|
|||
forUtil = new ForUtil(acceptableOverheadRatio, docOut);
|
||||
if (state.fieldInfos.hasProx()) {
|
||||
posDeltaBuffer = new int[MAX_DATA_SIZE];
|
||||
posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempBlockPostingsFormat.POS_EXTENSION),
|
||||
posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempBlockTreePostingsFormat.POS_EXTENSION),
|
||||
state.context);
|
||||
CodecUtil.writeHeader(posOut, POS_CODEC, VERSION_CURRENT);
|
||||
|
||||
|
@ -150,7 +150,7 @@ public final class TempPostingsWriter extends TempPostingsWriterBase {
|
|||
}
|
||||
|
||||
if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
|
||||
payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempBlockPostingsFormat.PAY_EXTENSION),
|
||||
payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempBlockTreePostingsFormat.PAY_EXTENSION),
|
||||
state.context);
|
||||
CodecUtil.writeHeader(payOut, PAY_CODEC, VERSION_CURRENT);
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.util.Arrays;
|
|||
|
||||
import org.apache.lucene.index.DocsEnum; // javadocs
|
||||
import org.apache.lucene.codecs.TempPostingsReaderBase; // javadocs
|
||||
import org.apache.lucene.index.OrdTermState;
|
||||
import org.apache.lucene.index.TermState;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
|
||||
|
@ -28,7 +29,7 @@ import org.apache.lucene.store.ByteArrayDataInput;
|
|||
* to produce a {@link DocsEnum} without re-seeking the
|
||||
* terms dict.
|
||||
*/
|
||||
public class TempTermState extends TermState {
|
||||
public class TempTermState extends OrdTermState {
|
||||
/** how many docs have this term */
|
||||
public int docFreq;
|
||||
/** total number of occurrences of this term */
|
||||
|
@ -36,6 +37,8 @@ public class TempTermState extends TermState {
|
|||
|
||||
/** the term's ord in the current block */
|
||||
public int termBlockOrd;
|
||||
/** fp into the terms dict primary file (_X.tim) that holds this term */
|
||||
public long blockFilePointer;
|
||||
|
||||
/** Sole constructor. (For invocation by subclass
|
||||
* constructors, typically implicit.) */
|
||||
|
@ -46,13 +49,15 @@ public class TempTermState extends TermState {
|
|||
public void copyFrom(TermState _other) {
|
||||
assert _other instanceof TempTermState : "can not copy from " + _other.getClass().getName();
|
||||
TempTermState other = (TempTermState) _other;
|
||||
super.copyFrom(_other);
|
||||
docFreq = other.docFreq;
|
||||
totalTermFreq = other.totalTermFreq;
|
||||
termBlockOrd = other.termBlockOrd;
|
||||
blockFilePointer = other.blockFilePointer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd;
|
||||
return "docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termBlockOrd=" + termBlockOrd + " blockFP=" + blockFilePointer;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -15,6 +15,6 @@
|
|||
|
||||
org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat
|
||||
org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat
|
||||
org.apache.lucene.codecs.temp.TempBlockPostingsFormat
|
||||
org.apache.lucene.codecs.temp.TempBlockTreePostingsFormat
|
||||
org.apache.lucene.codecs.temp.TempFSTPostingsFormat
|
||||
org.apache.lucene.codecs.temp.TempFSTOrdPostingsFormat
|
||||
|
|
Loading…
Reference in New Issue