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:
Han Jiang 2013-08-15 13:15:46 +00:00
parent 134cfe0b29
commit dd1070cd2b
11 changed files with 1442 additions and 58 deletions

View File

@ -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";
}

View File

@ -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");
}
}
}
}
}

View File

@ -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;
}
}
}

View File

@ -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

View File

@ -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,

View File

@ -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");

View File

@ -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,

View File

@ -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,

View File

@ -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);
}

View File

@ -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;
}
}

View File

@ -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