LUCENE-2872: block-encode terms in between indexed terms

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1061453 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2011-01-20 18:53:55 +00:00
parent f320ade2b7
commit 1a0f78778b
58 changed files with 1653 additions and 1163 deletions

View File

@ -144,9 +144,9 @@ public class InstantiatedTermsEnum extends TermsEnum {
}
@Override
public SeekStatus seek(BytesRef term, TermState state) throws IOException {
public void seek(BytesRef term, TermState state) throws IOException {
assert state != null && state instanceof OrdTermState;
return seek(((OrdTermState)state).ord); // just use the ord for simplicity
seek(((OrdTermState)state).ord); // just use the ord for simplicity
}
}

View File

@ -938,9 +938,9 @@ public class MemoryIndex implements Serializable {
}
@Override
public SeekStatus seek(BytesRef term, TermState state) throws IOException {
public void seek(BytesRef term, TermState state) throws IOException {
assert state != null;
return this.seek(((OrdTermState)state).ord);
this.seek(((OrdTermState)state).ord);
}
@Override

View File

@ -32,7 +32,7 @@ import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter;
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
@ -131,7 +131,7 @@ public class AppendingCodec extends Codec {
public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files)
throws IOException {
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
}

View File

@ -22,15 +22,15 @@ import java.util.Comparator;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
public class AppendingTermsDictReader extends PrefixCodedTermsReader {
public class AppendingTermsDictReader extends BlockTermsReader {
public AppendingTermsDictReader(TermsIndexReaderBase indexReader,
Directory dir, FieldInfos fieldInfos, String segment,
@ -43,7 +43,7 @@ public class AppendingTermsDictReader extends PrefixCodedTermsReader {
@Override
protected void readHeader(IndexInput in) throws IOException {
CodecUtil.checkHeader(in, AppendingTermsDictWriter.CODEC_NAME,
PrefixCodedTermsWriter.VERSION_START, PrefixCodedTermsWriter.VERSION_CURRENT);
BlockTermsWriter.VERSION_START, BlockTermsWriter.VERSION_CURRENT);
}
@Override

View File

@ -22,13 +22,13 @@ import java.util.Comparator;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
public class AppendingTermsDictWriter extends PrefixCodedTermsWriter {
public class AppendingTermsDictWriter extends BlockTermsWriter {
final static String CODEC_NAME = "APPENDING_TERMS_DICT";
public AppendingTermsDictWriter(TermsIndexWriterBase indexWriter,

View File

@ -215,7 +215,10 @@ public class TestParser extends LuceneTestCase {
}
private void dumpResults(String qType,Query q, int numDocs) throws IOException
{
TopDocs hits = searcher.search(q, null, numDocs);
if (VERBOSE) {
System.out.println("TEST: query=" + q);
}
TopDocs hits = searcher.search(q, null, numDocs);
assertTrue(qType +" should produce results ", hits.totalHits>0);
if(VERBOSE)
{

View File

@ -802,7 +802,7 @@ public class CheckIndex {
msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
} catch (Throwable e) {
msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
msg("ERROR: " + e);
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);

View File

@ -903,8 +903,7 @@ final class DocumentsWriter {
final static int BYTE_BLOCK_NOT_MASK = ~BYTE_BLOCK_MASK;
/* if you increase this, you must fix field cache impl for
* getTerms/getTermsIndex requires <= 32768. Also fix
* DeltaBytesWriter's TERM_EOF if necessary. */
* getTerms/getTermsIndex requires <= 32768. */
final static int MAX_TERM_LENGTH_UTF8 = BYTE_BLOCK_SIZE-2;
/* Initial chunks size of the shared int[] blocks used to

View File

@ -156,12 +156,12 @@ public class FilterIndexReader extends IndexReader {
}
@Override
public int docFreq() {
public int docFreq() throws IOException {
return in.docFreq();
}
@Override
public long totalTermFreq() {
public long totalTermFreq() throws IOException {
return in.totalTermFreq();
}
@ -181,8 +181,8 @@ public class FilterIndexReader extends IndexReader {
}
@Override
public SeekStatus seek(BytesRef term, TermState state) throws IOException {
return in.seek(term, state);
public void seek(BytesRef term, TermState state) throws IOException {
in.seek(term, state);
}
@Override

View File

@ -257,7 +257,7 @@ public final class MultiTermsEnum extends TermsEnum {
}
@Override
public int docFreq() {
public int docFreq() throws IOException {
int sum = 0;
for(int i=0;i<numTop;i++) {
sum += top[i].terms.docFreq();
@ -266,7 +266,7 @@ public final class MultiTermsEnum extends TermsEnum {
}
@Override
public long totalTermFreq() {
public long totalTermFreq() throws IOException {
long sum = 0;
for(int i=0;i<numTop;i++) {
final long v = top[i].terms.totalTermFreq();

View File

@ -100,11 +100,8 @@ public abstract class Terms {
* @see TermsEnum#seek(BytesRef, TermState) */
public DocsEnum docs(Bits skipDocs, BytesRef term, TermState termState, DocsEnum reuse) throws IOException {
final TermsEnum termsEnum = getThreadTermsEnum();
if (termsEnum.seek(term, termState) == TermsEnum.SeekStatus.FOUND) {
return termsEnum.docs(skipDocs, reuse);
} else {
return null;
}
termsEnum.seek(term, termState);
return termsEnum.docs(skipDocs, reuse);
}
/**
@ -116,11 +113,8 @@ public abstract class Terms {
* @see TermsEnum#seek(BytesRef, TermState) */
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, BytesRef term, TermState termState, DocsAndPositionsEnum reuse) throws IOException {
final TermsEnum termsEnum = getThreadTermsEnum();
if (termsEnum.seek(term, termState) == TermsEnum.SeekStatus.FOUND) {
return termsEnum.docsAndPositions(skipDocs, reuse);
} else {
return null;
}
termsEnum.seek(term, termState);
return termsEnum.docsAndPositions(skipDocs, reuse);
}
public long getUniqueTermCount() throws IOException {

View File

@ -80,7 +80,7 @@ public abstract class TermsEnum {
* use this method. Low-level implementations may position the TermsEnum
* without re-seeking the term dictionary.
* <p>
* Seeking by {@link TermState} should only be used iff the enu the state was
* Seeking by {@link TermState} should only be used iff the enum the state was
* obtained from and the enum the state is used for seeking are obtained from
* the same {@link IndexReader}, otherwise a {@link #seek(BytesRef, TermState)} call can
* leave the enum in undefined state.
@ -97,8 +97,8 @@ public abstract class TermsEnum {
* @param term the term the TermState corresponds to
* @param state the {@link TermState}
* */
public SeekStatus seek(BytesRef term, TermState state) throws IOException {
return seek(term);
public void seek(BytesRef term, TermState state) throws IOException {
seek(term);
}
/** Increments the enumeration to the next element.
@ -124,7 +124,7 @@ public abstract class TermsEnum {
* term. Do not call this before calling next() for the
* first time, after next() returns null or seek returns
* {@link SeekStatus#END}.*/
public abstract int docFreq();
public abstract int docFreq() throws IOException;
/** Returns the total number of occurrences of this term
* across all documents (the sum of the freq() for each
@ -132,7 +132,7 @@ public abstract class TermsEnum {
* codec doesn't support this measure. Note that, like
* other term measures, this measure does not take
* deleted documents into account. */
public abstract long totalTermFreq();
public abstract long totalTermFreq() throws IOException;
/** Get {@link DocsEnum} for the current term. Do not
* call this before calling {@link #next} or {@link
@ -242,7 +242,7 @@ public abstract class TermsEnum {
}
@Override
public SeekStatus seek(BytesRef term, TermState state) throws IOException {
public void seek(BytesRef term, TermState state) throws IOException {
throw new IllegalStateException("this method should never be called");
}
};

View File

@ -16,7 +16,6 @@ package org.apache.lucene.index.codecs;
* limitations under the License.
*/
import org.apache.lucene.index.DocsEnum; // javadocs
import org.apache.lucene.index.OrdTermState;
import org.apache.lucene.index.TermState;
@ -25,24 +24,32 @@ import org.apache.lucene.index.TermState;
* to produce a {@link DocsEnum} without re-seeking the
* terms dict.
*/
public class PrefixCodedTermState extends OrdTermState {
public int docFreq; // how many docs have this term
public long filePointer; // fp into the terms dict primary file (_X.tis)
public long totalTermFreq; // total number of occurrences of this term
public class BlockTermState extends OrdTermState {
public int docFreq; // how many docs have this term
public long totalTermFreq; // total number of occurrences of this term
public int termCount; // term ord are in the current block
public long blockFilePointer; // fp into the terms dict primary file (_X.tib) that holds this term
public int blockTermCount; // how many terms in current block
@Override
public void copyFrom(TermState _other) {
assert _other instanceof PrefixCodedTermState : "can not copy from " + _other.getClass().getName();
PrefixCodedTermState other = (PrefixCodedTermState) _other;
assert _other instanceof BlockTermState : "can not copy from " + _other.getClass().getName();
BlockTermState other = (BlockTermState) _other;
super.copyFrom(_other);
filePointer = other.filePointer;
docFreq = other.docFreq;
totalTermFreq = other.totalTermFreq;
termCount = other.termCount;
blockFilePointer = other.blockFilePointer;
// NOTE: don't copy blockTermCount;
// it's "transient": used only by the "primary"
// termState, and regenerated on seek by TermState
}
@Override
public String toString() {
return super.toString() + "[ord=" + ord + ", tis.filePointer=" + filePointer + ", docFreq=" + docFreq + ", totalTermFreq=" + totalTermFreq + "]";
return super.toString() + "ord=" + ord + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termCount=" + termCount + " blockFP=" + blockFilePointer;
}
}

View File

@ -0,0 +1,741 @@
package org.apache.lucene.index.codecs;
/**
* 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.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.Comparator;
import java.util.Iterator;
import java.util.TreeMap;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
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.index.codecs.standard.StandardPostingsReader; // javadocs
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
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.CodecUtil;
import org.apache.lucene.util.DoubleBarrelLRUCache;
/** Handles a terms dict, but decouples all details of
* doc/freqs/positions reading to an instance of {@link
* PostingsReaderBase}. 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 BlockTermsReader 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 PostingsReaderBase postingsReader;
private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
// Comparator that orders our terms
private final Comparator<BytesRef> termComp;
// Caches the most recently looked-up field + terms:
private final DoubleBarrelLRUCache<FieldAndTerm,BlockTermState> termsCache;
// Reads the terms index
private TermsIndexReaderBase indexReader;
// keeps the dirStart offset
protected long dirOffset;
// 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 = new BytesRef(other.term);
}
@Override
public boolean equals(Object _other) {
FieldAndTerm other = (FieldAndTerm) _other;
return other.field == field && term.bytesEquals(other.term);
}
@Override
public Object clone() {
return new FieldAndTerm(this);
}
@Override
public int hashCode() {
return field.hashCode() * 31 + term.hashCode();
}
}
private String segment;
public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, int readBufferSize,
Comparator<BytesRef> termComp, int termsCacheSize, String codecId)
throws IOException {
this.postingsReader = postingsReader;
termsCache = new DoubleBarrelLRUCache<FieldAndTerm,BlockTermState>(termsCacheSize);
this.termComp = termComp;
this.segment = segment;
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
readBufferSize);
boolean success = false;
try {
readHeader(in);
// Have PostingsReader init itself
postingsReader.init(in);
// Read per-field details
seekDir(in, dirOffset);
final int numFields = in.readVInt();
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.omitTermFreqAndPositions ? -1 : in.readVLong();
assert !fields.containsKey(fieldInfo.name);
fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, termsStartPointer, sumTotalTermFreq));
}
success = true;
} finally {
if (!success) {
in.close();
}
}
this.indexReader = indexReader;
}
protected void readHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(in, BlockTermsWriter.CODEC_NAME,
BlockTermsWriter.VERSION_START,
BlockTermsWriter.VERSION_CURRENT);
dirOffset = in.readLong();
}
protected void seekDir(IndexInput input, long dirOffset)
throws IOException {
input.seek(dirOffset);
}
@Override
public void loadTermsIndex(int indexDivisor) throws IOException {
indexReader.loadTermsIndex(indexDivisor);
}
@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 {
try {
if (postingsReader != null) {
postingsReader.close();
}
} finally {
for(FieldReader field : fields.values()) {
field.close();
}
}
}
}
public static void files(Directory dir, SegmentInfo segmentInfo, String id, Collection<String> files) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, BlockTermsWriter.TERMS_EXTENSION));
}
public static void getExtensions(Collection<String> extensions) {
extensions.add(BlockTermsWriter.TERMS_EXTENSION);
}
@Override
public FieldsEnum iterator() {
return new TermFieldsEnum();
}
@Override
public Terms terms(String field) throws IOException {
return fields.get(field);
}
// Iterates through all fields
private class TermFieldsEnum extends FieldsEnum {
final Iterator<FieldReader> it;
FieldReader current;
TermFieldsEnum() {
it = fields.values().iterator();
}
@Override
public String next() {
if (it.hasNext()) {
current = it.next();
return current.fieldInfo.name;
} else {
current = null;
return null;
}
}
@Override
public TermsEnum terms() throws IOException {
return current.iterator();
}
}
private class FieldReader extends Terms implements Closeable {
final long numTerms;
final FieldInfo fieldInfo;
final long termsStartPointer;
final long sumTotalTermFreq;
FieldReader(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq) {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
this.numTerms = numTerms;
this.termsStartPointer = termsStartPointer;
this.sumTotalTermFreq = sumTotalTermFreq;
}
@Override
public Comparator<BytesRef> getComparator() {
return termComp;
}
@Override
public void close() {
super.close();
}
@Override
public TermsEnum iterator() throws IOException {
return new SegmentTermsEnum();
}
@Override
public long getUniqueTermCount() {
return numTerms;
}
@Override
public long getSumTotalTermFreq() {
return sumTotalTermFreq;
}
// Iterates through terms in this field
private final class SegmentTermsEnum extends TermsEnum {
private final IndexInput in;
private final BlockTermState 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 seek(TermState), do defer seeking. If the app then
calls next() (which is not "typical"), then we'll do the real seek */
private boolean seekPending;
/* How many blocks we've read since last seek. Once this
is >= indexEnum.getDivisor() we set indexIsCurrent to false (since
the index can no long bracket seek-within-block). */
private int blocksSinceSeek;
private byte[] termSuffixes;
private ByteArrayDataInput termSuffixesReader = new ByteArrayDataInput(null);
/* Common prefix used for all terms in this block. */
private int termBlockPrefix;
private byte[] docFreqBytes;
private final ByteArrayDataInput freqReader = new ByteArrayDataInput(null);
private int metaDataUpto;
public SegmentTermsEnum() throws IOException {
in = (IndexInput) BlockTermsReader.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);
}
@Override
public Comparator<BytesRef> getComparator() {
return termComp;
}
@Override
public SeekStatus seek(final BytesRef target, final boolean useCache) 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());
}
}
*/
// Check cache
if (useCache) {
fieldTerm.term = target;
// TODO: should we differentiate "frozen"
// TermState (ie one that was cloned and
// cached/returned by termState()) from the
// malleable (primary) one?
final TermState cachedState = termsCache.get(fieldTerm);
if (cachedState != null) {
seekPending = true;
//System.out.println(" cached!");
seek(target, cachedState);
//System.out.println(" term=" + term.utf8ToString());
return SeekStatus.FOUND;
}
}
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 = termComp.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 || termComp.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;
blocksSinceSeek = 0;
if (doOrd) {
state.ord = indexEnum.ord()-1;
}
// NOTE: the first _next() after an index seek is
// a bit wasteful, since it redundantly reads some
// suffix bytes into the buffer. We could avoid storing
// those bytes in the primary file, but then when
// next()ing over an index term we'd have to
// special case it:
term.copy(indexEnum.term());
//System.out.println(" seek: term=" + term.utf8ToString());
} else {
////System.out.println(" skip seek");
}
seekPending = false;
// Now scan:
while (_next() != null) {
final int cmp = termComp.compare(term, target);
if (cmp == 0) {
// Match!
if (useCache) {
// Store in cache
decodeMetaData();
termsCache.put(new FieldAndTerm(fieldTerm), (BlockTermState) state.clone());
}
//System.out.println(" FOUND");
return SeekStatus.FOUND;
} else if (cmp > 0) {
//System.out.println(" NOT_FOUND term=" + term.utf8ToString());
return SeekStatus.NOT_FOUND;
}
// 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;
}
indexIsCurrent = false;
//System.out.println(" END");
return SeekStatus.END;
}
@Override
public BytesRef next() throws IOException {
//System.out.println("BTR.next() seekPending=" + seekPending + " pendingSeekCount=" + state.termCount);
// 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.termCount;
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.termCount < 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 this=" + this + " termCount=" + state.termCount + " (vs " + state.blockTermCount + ")");
if (state.termCount == state.blockTermCount) {
if (!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.termCount++;
// NOTE: meaningless in the non-ord case
state.ord++;
//System.out.println(" return term=" + fieldInfo.name + ":" + term.utf8ToString() + " " + term);
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 skipDocs, DocsEnum reuse) throws IOException {
//System.out.println("BTR.docs this=" + this);
decodeMetaData();
//System.out.println(" state.docFreq=" + state.docFreq);
final DocsEnum docsEnum = postingsReader.docs(fieldInfo, state, skipDocs, reuse);
assert docsEnum != null;
return docsEnum;
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
//System.out.println("BTR.d&p this=" + this);
decodeMetaData();
if (fieldInfo.omitTermFreqAndPositions) {
return null;
} else {
DocsAndPositionsEnum dpe = postingsReader.docsAndPositions(fieldInfo, state, skipDocs, reuse);
//System.out.println(" return d&pe=" + dpe);
return dpe;
}
}
@Override
public void seek(BytesRef target, TermState otherState) throws IOException {
//System.out.println("BTR.seek termState target=" + target.utf8ToString() + " " + target + " this=" + this);
assert otherState != null && otherState instanceof BlockTermState;
assert !doOrd || ((BlockTermState) otherState).ord < numTerms;
state.copyFrom(otherState);
seekPending = true;
indexIsCurrent = false;
term.copy(target);
}
@Override
public TermState termState() throws IOException {
//System.out.println("BTR.termState this=" + this);
decodeMetaData();
TermState ts = (TermState) state.clone();
//System.out.println(" return ts=" + ts);
return ts;
}
@Override
public SeekStatus seek(long ord) throws IOException {
//System.out.println("BTR.seek by ord ord=" + ord);
if (indexEnum == null) {
throw new IllegalStateException("terms index was not loaded");
}
if (ord >= numTerms) {
state.ord = numTerms-1;
return SeekStatus.END;
}
// 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;
blocksSinceSeek = 0;
seekPending = false;
state.ord = indexEnum.ord()-1;
assert state.ord >= -1: "ord=" + state.ord;
term.copy(indexEnum.term());
// Now, scan:
int left = (int) (ord - state.ord);
while(left > 0) {
final BytesRef term = _next();
assert term != null;
left--;
assert indexIsCurrent;
}
// always found
return SeekStatus.FOUND;
}
public long ord() {
if (!doOrd) {
throw new UnsupportedOperationException();
}
return state.ord;
}
private void doPendingSeek() {
}
/* 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();
state.blockTermCount = in.readVInt();
//System.out.println(" blockTermCount=" + state.blockTermCount);
if (state.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);
// 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);
metaDataUpto = 0;
state.termCount = 0;
postingsReader.readTermsBlock(in, fieldInfo, state);
blocksSinceSeek++;
indexIsCurrent &= (blocksSinceSeek < indexReader.getDivisor());
//System.out.println(" indexIsCurrent=" + indexIsCurrent);
return true;
}
private void decodeMetaData() throws IOException {
//System.out.println("BTR.decodeMetadata mdUpto=" + metaDataUpto + " vs termCount=" + state.termCount + " state=" + state);
if (!seekPending) {
// lazily catch up on metadata decode:
final int limit = state.termCount;
state.termCount = metaDataUpto;
while (metaDataUpto < limit) {
//System.out.println(" decode");
// 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
state.docFreq = freqReader.readVInt();
if (!fieldInfo.omitTermFreqAndPositions) {
state.totalTermFreq = state.docFreq + freqReader.readVLong();
}
postingsReader.nextTerm(fieldInfo, state);
metaDataUpto++;
state.termCount++;
}
} else {
//System.out.println(" skip! seekPending");
}
}
}
}
}

View File

@ -19,31 +19,35 @@ package org.apache.lucene.index.codecs;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.BytesRef;
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.CodecUtil;
import org.apache.lucene.util.RamUsageEstimator;
// 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 and interacts with docs/positions
* consumers to write the postings files.
* Writes terms dict, block-encoding (column stride) each
* term's metadata for each set of terms between two
* index terms.
*
* The [new] terms dict format is field-centric: each field
* has its own section in the file. Fields are written in
* UTF16 string comparison order. Within each field, each
* term's text is written in UTF16 string comparison order.
* @lucene.experimental
*/
public class PrefixCodedTermsWriter extends FieldsConsumer {
public class BlockTermsWriter extends FieldsConsumer {
final static String CODEC_NAME = "STANDARD_TERMS_DICT";
final static String CODEC_NAME = "BLOCK_TERMS_DICT";
// Initial format
public static final int VERSION_START = 0;
@ -51,9 +55,7 @@ public class PrefixCodedTermsWriter extends FieldsConsumer {
public static final int VERSION_CURRENT = VERSION_START;
/** Extension of terms file */
static final String TERMS_EXTENSION = "tis";
private final DeltaBytesWriter termWriter;
static final String TERMS_EXTENSION = "tib";
protected final IndexOutput out;
final PostingsWriterBase postingsWriter;
@ -62,8 +64,9 @@ public class PrefixCodedTermsWriter extends FieldsConsumer {
private final TermsIndexWriterBase termsIndexWriter;
private final List<TermsWriter> fields = new ArrayList<TermsWriter>();
private final Comparator<BytesRef> termComp;
private final String segment;
public PrefixCodedTermsWriter(
public BlockTermsWriter(
TermsIndexWriterBase termsIndexWriter,
SegmentWriteState state,
PostingsWriterBase postingsWriter,
@ -73,19 +76,18 @@ public class PrefixCodedTermsWriter extends FieldsConsumer {
this.termsIndexWriter = termsIndexWriter;
this.termComp = termComp;
out = state.directory.createOutput(termsFileName);
termsIndexWriter.setTermsOutput(out);
fieldInfos = state.fieldInfos;
writeHeader(out);
termWriter = new DeltaBytesWriter(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
}
protected void writeHeader(IndexOutput out) throws IOException {
// Count indexed fields up front
CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT);
out.writeLong(0); // leave space for end index pointer
@ -93,14 +95,15 @@ public class PrefixCodedTermsWriter extends FieldsConsumer {
@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);
TermsIndexWriterBase.FieldWriter fieldIndexWriter = termsIndexWriter.addField(field, out.getFilePointer());
final TermsWriter terms = new TermsWriter(fieldIndexWriter, field, postingsWriter);
fields.add(terms);
return terms;
}
@Override
public void close() throws IOException {
@ -146,6 +149,11 @@ public class PrefixCodedTermsWriter extends FieldsConsumer {
out.writeLong(dirStart);
}
private static class TermEntry {
public final BytesRef term = new BytesRef();
public TermStats stats;
}
class TermsWriter extends TermsConsumer {
private final FieldInfo fieldInfo;
private final PostingsWriterBase postingsWriter;
@ -153,6 +161,11 @@ public class PrefixCodedTermsWriter extends FieldsConsumer {
private long numTerms;
private final TermsIndexWriterBase.FieldWriter fieldIndexWriter;
long sumTotalTermFreq;
private final BytesRef lastTerm = new BytesRef();
private TermEntry[] pendingTerms;
private int pendingCount;
TermsWriter(
TermsIndexWriterBase.FieldWriter fieldIndexWriter,
@ -161,8 +174,10 @@ public class PrefixCodedTermsWriter extends FieldsConsumer {
{
this.fieldInfo = fieldInfo;
this.fieldIndexWriter = fieldIndexWriter;
termWriter.reset();
pendingTerms = new TermEntry[32];
for(int i=0;i<pendingTerms.length;i++) {
pendingTerms[i] = new TermEntry();
}
termsStartPointer = out.getFilePointer();
postingsWriter.setField(fieldInfo);
this.postingsWriter = postingsWriter;
@ -175,48 +190,127 @@ public class PrefixCodedTermsWriter extends FieldsConsumer {
@Override
public PostingsConsumer startTerm(BytesRef text) throws IOException {
//System.out.println("BTW.startTerm seg=" + segment + " term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text);
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("finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " fp=" + out.getFilePointer());
//System.out.println("BTW.finishTerm seg=" + segment + " term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " df=" + stats.docFreq);
final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, stats);
termWriter.write(text);
final int highBit = isIndexTerm ? 0x80 : 0;
//System.out.println(" isIndex=" + isIndexTerm);
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());
}
// This is a vInt, except, we steal top bit to record
// whether this was an indexed term:
if ((stats.docFreq & ~0x3F) == 0) {
// Fast case -- docFreq fits in 6 bits
out.writeByte((byte) (highBit | stats.docFreq));
} else {
// Write bottom 6 bits of docFreq, then write the
// remainder as vInt:
out.writeByte((byte) (highBit | 0x40 | (stats.docFreq & 0x3F)));
out.writeVInt(stats.docFreq >>> 6);
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;
}
if (!fieldInfo.omitTermFreqAndPositions) {
assert stats.totalTermFreq >= stats.docFreq;
out.writeVLong(stats.totalTermFreq - stats.docFreq);
}
postingsWriter.finishTerm(stats, isIndexTerm);
final TermEntry te = pendingTerms[pendingCount];
te.term.copy(text);
te.stats = stats;
pendingCount++;
postingsWriter.finishTerm(stats);
numTerms++;
}
// Finishes all terms in this field
@Override
public void finish(long sumTotalTermFreq) throws IOException {
if (pendingCount > 0) {
flushBlock();
}
// EOF marker:
out.writeVInt(0);
this.sumTotalTermFreq = sumTotalTermFreq;
out.writeVInt(DeltaBytesWriter.TERM_EOF);
fieldIndexWriter.finish();
fieldIndexWriter.finish(out.getFilePointer());
}
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 pendingCount=" + pendingCount);
// 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.omitTermFreqAndPositions) {
bytesWriter.writeVLong(stats.totalTermFreq-stats.docFreq);
}
}
out.writeVInt((int) bytesWriter.getFilePointer());
bytesWriter.writeTo(out);
bytesWriter.reset();
postingsWriter.flushTermsBlock();
lastPrevTerm.copy(pendingTerms[pendingCount-1].term);
pendingCount = 0;
}
}
}

View File

@ -1,52 +0,0 @@
package org.apache.lucene.index.codecs;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
// Handles reading incremental UTF8 encoded terms
final class DeltaBytesReader {
final BytesRef term = new BytesRef();
final IndexInput in;
DeltaBytesReader(IndexInput in) {
this.in = in;
term.bytes = new byte[10];
}
void reset(BytesRef text) {
term.copy(text);
}
boolean read() throws IOException {
final int start = in.readVInt();
if (start == DeltaBytesWriter.TERM_EOF) {
return false;
}
final int suffix = in.readVInt();
assert start <= term.length: "start=" + start + " length=" + term.length;
final int newLength = start+suffix;
term.grow(newLength);
in.readBytes(term.bytes, start, suffix);
term.length = newLength;
return true;
}
}

View File

@ -1,75 +0,0 @@
package org.apache.lucene.index.codecs;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
import java.io.IOException;
final class DeltaBytesWriter {
// Must be bigger than
// DocumentsWriter.MAX_TERM_LENGTH_UTF8. If you change
// this it's an index format change, so that change must be
// versioned:
final static int TERM_EOF = BYTE_BLOCK_SIZE;
private byte[] lastBytes = new byte[10];
private int lastLength;
final IndexOutput out;
DeltaBytesWriter(IndexOutput out) {
this.out = out;
}
void reset() {
lastLength = 0;
}
void write(BytesRef text) throws IOException {
int start = 0;
int upto = text.offset;
final int length = text.length;
final byte[] bytes = text.bytes;
final int limit = length < lastLength ? length : lastLength;
while(start < limit) {
if (bytes[upto] != lastBytes[start]) {
break;
}
start++;
upto++;
}
final int suffix = length - start;
out.writeVInt(start); // prefix
out.writeVInt(suffix); // suffix
out.writeBytes(bytes, upto, suffix);
if (lastBytes.length < length) {
lastBytes = ArrayUtil.grow(lastBytes, length);
}
// TODO: is this copy really necessary? I don't think
// caller actually modifies these bytes, so we can save
// by reference?
System.arraycopy(bytes, upto, lastBytes, start, suffix);
lastLength = length;
}
}

View File

@ -94,6 +94,7 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
// Read directory
final int numFields = in.readVInt();
//System.out.println("FGR: init seg=" + segment + " div=" + indexDivisor + " nF=" + numFields);
for(int i=0;i<numFields;i++) {
final int field = in.readVInt();
final int numIndexTerms = in.readVInt();
@ -241,9 +242,6 @@ public class FixedGapTermsIndexReader extends TermsIndexReaderBase {
this.packedOffsetsStart = packedOffsetsStart;
this.numIndexTerms = numIndexTerms;
// We still create the indexReader when indexDivisor
// is -1, so that PrefixCodedTermsReader can call
// isIndexTerm for each field:
if (indexDivisor > 0) {
loadTermsIndex();
}

View File

@ -53,7 +53,6 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
private final List<SimpleFieldWriter> fields = new ArrayList<SimpleFieldWriter>();
private final FieldInfos fieldInfos; // unread
private IndexOutput termsOut;
public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException {
final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
@ -71,13 +70,9 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
}
@Override
public void setTermsOutput(IndexOutput termsOut) {
this.termsOut = termsOut;
}
@Override
public FieldWriter addField(FieldInfo field) {
SimpleFieldWriter writer = new SimpleFieldWriter(field);
public FieldWriter addField(FieldInfo field, long termsFilePointer) {
//System.out.println("FGW: addFfield=" + field.name);
SimpleFieldWriter writer = new SimpleFieldWriter(field, termsFilePointer);
fields.add(writer);
return writer;
}
@ -119,10 +114,10 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
private final BytesRef lastTerm = new BytesRef();
SimpleFieldWriter(FieldInfo fieldInfo) {
SimpleFieldWriter(FieldInfo fieldInfo, long termsFilePointer) {
this.fieldInfo = fieldInfo;
indexStart = out.getFilePointer();
termsStart = lastTermsPointer = termsOut.getFilePointer();
termsStart = lastTermsPointer = termsFilePointer;
termLengths = new short[0];
termsPointerDeltas = new int[0];
}
@ -130,33 +125,8 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
@Override
public boolean checkIndexTerm(BytesRef text, TermStats stats) throws IOException {
// First term is first indexed term:
//System.out.println("FGW: checkIndexTerm text=" + text.utf8ToString());
if (0 == (numTerms++ % termIndexInterval)) {
final int indexedTermLength = indexedTermPrefixLength(lastTerm, text);
// write only the min prefix that shows the diff
// against prior term
out.writeBytes(text.bytes, text.offset, indexedTermLength);
if (termLengths.length == numIndexTerms) {
termLengths = ArrayUtil.grow(termLengths);
}
if (termsPointerDeltas.length == numIndexTerms) {
termsPointerDeltas = ArrayUtil.grow(termsPointerDeltas);
}
// save delta terms pointer
final long fp = termsOut.getFilePointer();
termsPointerDeltas[numIndexTerms] = (int) (fp - lastTermsPointer);
lastTermsPointer = fp;
// save term length (in bytes)
assert indexedTermLength <= Short.MAX_VALUE;
termLengths[numIndexTerms] = (short) indexedTermLength;
totTermLength += indexedTermLength;
lastTerm.copy(text);
numIndexTerms++;
return true;
} else {
if (0 == numTerms % termIndexInterval) {
@ -169,13 +139,41 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
}
@Override
public void finish() throws IOException {
public void add(BytesRef text, TermStats stats, long termsFilePointer) throws IOException {
final int indexedTermLength = indexedTermPrefixLength(lastTerm, text);
//System.out.println("FGW: add text=" + text.utf8ToString() + " " + text + " fp=" + termsFilePointer);
// write only the min prefix that shows the diff
// against prior term
out.writeBytes(text.bytes, text.offset, indexedTermLength);
if (termLengths.length == numIndexTerms) {
termLengths = ArrayUtil.grow(termLengths);
}
if (termsPointerDeltas.length == numIndexTerms) {
termsPointerDeltas = ArrayUtil.grow(termsPointerDeltas);
}
// save delta terms pointer
termsPointerDeltas[numIndexTerms] = (int) (termsFilePointer - lastTermsPointer);
lastTermsPointer = termsFilePointer;
// save term length (in bytes)
assert indexedTermLength <= Short.MAX_VALUE;
termLengths[numIndexTerms] = (short) indexedTermLength;
totTermLength += indexedTermLength;
lastTerm.copy(text);
numIndexTerms++;
}
@Override
public void finish(long termsFilePointer) throws IOException {
// write primary terms dict offsets
packedIndexStart = out.getFilePointer();
final long maxValue = termsOut.getFilePointer();
PackedInts.Writer w = PackedInts.getWriter(out, numIndexTerms, PackedInts.bitsRequired(maxValue));
PackedInts.Writer w = PackedInts.getWriter(out, numIndexTerms, PackedInts.bitsRequired(termsFilePointer));
// relative to our indexStart
long upto = 0;

View File

@ -28,12 +28,12 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.index.codecs.standard.StandardPostingsWriter; // javadocs
/** PrefixCodedTermsReader interacts with a single instance
* of this to manage creation of {@link DocsEnum} and
/** BlockTermsReader interacts with a single instance
* of this class to manage creation of {@link DocsEnum} and
* {@link DocsAndPositionsEnum} instances. It provides an
* IndexInput (termsIn) where this class may read any
* previously stored data that it had written in its
* corresponding {@link StandardPostingsWriter} at indexing
* corresponding {@link PostingsWriterBase} at indexing
* time.
* @lucene.experimental */
@ -42,17 +42,23 @@ public abstract class PostingsReaderBase implements Closeable {
public abstract void init(IndexInput termsIn) throws IOException;
/** Return a newly created empty TermState */
public abstract PrefixCodedTermState newTermState() throws IOException;
public abstract BlockTermState newTermState() throws IOException;
public abstract void readTerm(IndexInput termsIn, FieldInfo fieldInfo, PrefixCodedTermState state, boolean isIndexTerm) throws IOException;
/** Actually decode metadata for next term */
public abstract void nextTerm(FieldInfo fieldInfo, BlockTermState state) throws IOException;
/** Must fully consume state, since after this call that
* TermState may be reused. */
public abstract DocsEnum docs(FieldInfo fieldInfo, PrefixCodedTermState state, Bits skipDocs, DocsEnum reuse) throws IOException;
public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsEnum reuse) throws IOException;
/** Must fully consume state, since after this call that
* TermState may be reused. */
public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, PrefixCodedTermState state, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException;
public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException;
public abstract void close() throws IOException;
/** Reads data for all terms in the next block; this
* method should merely load the byte[] blob but not
* decode, which is done in {@link #nextTerm}. */
public abstract void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState termState) throws IOException;
}

View File

@ -33,8 +33,10 @@ public abstract class PostingsWriterBase extends PostingsConsumer implements Clo
public abstract void startTerm() throws IOException;
public abstract void flushTermsBlock() throws IOException;
/** Finishes the current term */
public abstract void finishTerm(TermStats stats, boolean isIndexTerm) throws IOException;
public abstract void finishTerm(TermStats stats) throws IOException;
public abstract void setField(FieldInfo fieldInfo);

View File

@ -1,607 +0,0 @@
package org.apache.lucene.index.codecs;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.io.Closeable;
import java.util.Collection;
import java.util.Iterator;
import java.util.TreeMap;
import java.util.Comparator;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
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.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.DoubleBarrelLRUCache;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.index.codecs.standard.StandardPostingsReader; // javadocs
/** Handles a terms dict, but decouples all details of
* doc/freqs/positions reading to an instance of {@link
* StandardPostingsReader}. 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 PrefixCodedTermsReader 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 PostingsReaderBase postingsReader;
private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
// Comparator that orders our terms
private final Comparator<BytesRef> termComp;
// Caches the most recently looked-up field + terms:
private final DoubleBarrelLRUCache<FieldAndTerm,PrefixCodedTermState> termsCache;
// Reads the terms index
private TermsIndexReaderBase indexReader;
// keeps the dirStart offset
protected long dirOffset;
// 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 = new BytesRef(other.term);
}
@Override
public boolean equals(Object _other) {
FieldAndTerm other = (FieldAndTerm) _other;
return other.field == field && term.bytesEquals(other.term);
}
@Override
public Object clone() {
return new FieldAndTerm(this);
}
@Override
public int hashCode() {
return field.hashCode() * 31 + term.hashCode();
}
}
public PrefixCodedTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, int readBufferSize,
Comparator<BytesRef> termComp, int termsCacheSize, String codecId)
throws IOException {
this.postingsReader = postingsReader;
termsCache = new DoubleBarrelLRUCache<FieldAndTerm,PrefixCodedTermState>(termsCacheSize);
this.termComp = termComp;
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, PrefixCodedTermsWriter.TERMS_EXTENSION),
readBufferSize);
boolean success = false;
try {
readHeader(in);
// Have PostingsReader init itself
postingsReader.init(in);
// Read per-field details
seekDir(in, dirOffset);
final int numFields = in.readVInt();
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.omitTermFreqAndPositions ? -1 : in.readVLong();
assert !fields.containsKey(fieldInfo.name);
fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, termsStartPointer, sumTotalTermFreq));
}
success = true;
} finally {
if (!success) {
in.close();
}
}
this.indexReader = indexReader;
}
protected void readHeader(IndexInput input) throws IOException {
CodecUtil.checkHeader(in, PrefixCodedTermsWriter.CODEC_NAME,
PrefixCodedTermsWriter.VERSION_START, PrefixCodedTermsWriter.VERSION_CURRENT);
dirOffset = in.readLong();
}
protected void seekDir(IndexInput input, long dirOffset)
throws IOException {
input.seek(dirOffset);
}
@Override
public void loadTermsIndex(int indexDivisor) throws IOException {
indexReader.loadTermsIndex(indexDivisor);
}
@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 {
try {
if (postingsReader != null) {
postingsReader.close();
}
} finally {
for(FieldReader field : fields.values()) {
field.close();
}
}
}
}
public static void files(Directory dir, SegmentInfo segmentInfo, String id, Collection<String> files) {
files.add(IndexFileNames.segmentFileName(segmentInfo.name, id, PrefixCodedTermsWriter.TERMS_EXTENSION));
}
public static void getExtensions(Collection<String> extensions) {
extensions.add(PrefixCodedTermsWriter.TERMS_EXTENSION);
}
@Override
public FieldsEnum iterator() {
return new TermFieldsEnum();
}
@Override
public Terms terms(String field) throws IOException {
return fields.get(field);
}
// Iterates through all fields
private class TermFieldsEnum extends FieldsEnum {
final Iterator<FieldReader> it;
FieldReader current;
TermFieldsEnum() {
it = fields.values().iterator();
}
@Override
public String next() {
if (it.hasNext()) {
current = it.next();
return current.fieldInfo.name;
} else {
current = null;
return null;
}
}
@Override
public TermsEnum terms() throws IOException {
return current.iterator();
}
}
private class FieldReader extends Terms implements Closeable {
final long numTerms;
final FieldInfo fieldInfo;
final long termsStartPointer;
final long sumTotalTermFreq;
FieldReader(FieldInfo fieldInfo, long numTerms, long termsStartPointer, long sumTotalTermFreq) {
assert numTerms > 0;
this.fieldInfo = fieldInfo;
this.numTerms = numTerms;
this.termsStartPointer = termsStartPointer;
this.sumTotalTermFreq = sumTotalTermFreq;
}
@Override
public Comparator<BytesRef> getComparator() {
return termComp;
}
@Override
public void close() {
super.close();
}
@Override
public TermsEnum iterator() throws IOException {
return new SegmentTermsEnum();
}
@Override
public long getUniqueTermCount() {
return numTerms;
}
@Override
public long getSumTotalTermFreq() {
return sumTotalTermFreq;
}
// Iterates through terms in this field, not supporting ord()
private final class SegmentTermsEnum extends TermsEnum {
private final IndexInput in;
private final DeltaBytesReader bytesReader;
private final PrefixCodedTermState state;
private boolean seekPending;
private final FieldAndTerm fieldTerm = new FieldAndTerm();
private final TermsIndexReaderBase.FieldIndexEnum indexEnum;
private boolean positioned;
private boolean didIndexNext;
private BytesRef nextIndexTerm;
private boolean isIndexTerm;
private final boolean doOrd;
SegmentTermsEnum() throws IOException {
in = (IndexInput) PrefixCodedTermsReader.this.in.clone();
in.seek(termsStartPointer);
indexEnum = indexReader.getFieldEnum(fieldInfo);
doOrd = indexReader.supportsOrd();
bytesReader = new DeltaBytesReader(in);
fieldTerm.field = fieldInfo.name;
state = postingsReader.newTermState();
state.totalTermFreq = -1;
state.ord = -1;
}
@Override
public Comparator<BytesRef> getComparator() {
return termComp;
}
// called only from assert
private boolean first;
private int indexTermCount;
private boolean startSeek() {
first = true;
indexTermCount = 0;
return true;
}
private boolean checkSeekScan() {
if (!first && isIndexTerm) {
indexTermCount++;
if (indexTermCount >= indexReader.getDivisor()) {
//System.out.println("now fail count=" + indexTermCount);
return false;
}
}
first = false;
return true;
}
/** Seeks until the first term that's >= the provided
* text; returns SeekStatus.FOUND if the exact term
* is found, SeekStatus.NOT_FOUND if a different term
* was found, SeekStatus.END if we hit EOF */
@Override
public SeekStatus seek(final BytesRef term, final boolean useCache) throws IOException {
if (indexEnum == null) {
throw new IllegalStateException("terms index was not loaded");
}
//System.out.println("te.seek term=" + fieldInfo.name + ":" + term.utf8ToString() + " current=" + term().utf8ToString() + " useCache=" + useCache + " this=" + this);
// Check cache
fieldTerm.term = term;
TermState cachedState;
if (useCache) {
cachedState = termsCache.get(fieldTerm);
if (cachedState != null) {
state.copyFrom(cachedState);
setTermState(term, state);
positioned = false;
//System.out.println(" cached!");
return SeekStatus.FOUND;
}
} else {
cachedState = null;
}
boolean doSeek = true;
if (positioned) {
final int cmp = termComp.compare(bytesReader.term, term);
if (cmp == 0) {
// already at the requested term
return SeekStatus.FOUND;
} else if (cmp < 0) {
if (seekPending) {
seekPending = false;
in.seek(state.filePointer);
indexEnum.seek(bytesReader.term);
didIndexNext = false;
}
// 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 || termComp.compare(term, nextIndexTerm) < 0) {
// Optimization: requested term is within the
// same index block we are now in; skip seeking
// (but do scanning):
doSeek = false;
//System.out.println(" skip seek: nextIndexTerm=" + nextIndexTerm);
}
}
}
if (doSeek) {
positioned = true;
// Ask terms index to find biggest index term that's <=
// our text:
in.seek(indexEnum.seek(term));
didIndexNext = false;
if (doOrd) {
state.ord = indexEnum.ord()-1;
}
seekPending = false;
// NOTE: the first next() after an index seek is
// wasteful, since it redundantly reads the same
// bytes into the buffer. We could avoid storing
// those bytes in the primary file, but then when
// scanning over an index term we'd have to
// special case it:
bytesReader.reset(indexEnum.term());
//System.out.println(" doSeek term=" + indexEnum.term().utf8ToString() + " vs target=" + term.utf8ToString());
} else {
//System.out.println(" skip seek");
}
assert startSeek();
// Now scan:
while (next() != null) {
final int cmp = termComp.compare(bytesReader.term, term);
if (cmp == 0) {
// Done!
if (useCache) {
cacheTerm(fieldTerm);
}
return SeekStatus.FOUND;
} else if (cmp > 0) {
return SeekStatus.NOT_FOUND;
}
// 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 checkSeekScan();
}
positioned = false;
return SeekStatus.END;
}
private final void setTermState(BytesRef term, final TermState termState) {
assert termState != null && termState instanceof PrefixCodedTermState;
state.copyFrom(termState);
seekPending = true;
bytesReader.term.copy(term);
}
private final void cacheTerm(FieldAndTerm other) {
// Store in cache
final FieldAndTerm entryKey = new FieldAndTerm(other);
final PrefixCodedTermState cachedState = (PrefixCodedTermState) state.clone();
// this is fp after current term
cachedState.filePointer = in.getFilePointer();
termsCache.put(entryKey, cachedState);
}
@Override
public BytesRef term() {
return bytesReader.term;
}
@Override
public BytesRef next() throws IOException {
if (seekPending) {
seekPending = false;
in.seek(state.filePointer);
indexEnum.seek(bytesReader.term);
didIndexNext = false;
}
if (!bytesReader.read()) {
//System.out.println("te.next end!");
positioned = false;
return null;
}
final byte b = in.readByte();
isIndexTerm = (b & 0x80) != 0;
if ((b & 0x40) == 0) {
// Fast case -- docFreq fits in 6 bits
state.docFreq = b & 0x3F;
} else {
state.docFreq = (in.readVInt() << 6) | (b & 0x3F);
}
if (!fieldInfo.omitTermFreqAndPositions) {
state.totalTermFreq = state.docFreq + in.readVLong();
}
postingsReader.readTerm(in,
fieldInfo, state,
isIndexTerm);
if (doOrd) {
state.ord++;
}
positioned = true;
//System.out.println("te.next term=" + bytesReader.term.utf8ToString());
return bytesReader.term;
}
@Override
public int docFreq() {
return state.docFreq;
}
@Override
public long totalTermFreq() {
return state.totalTermFreq;
}
@Override
public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
final DocsEnum docsEnum = postingsReader.docs(fieldInfo, state, skipDocs, reuse);
assert docsEnum != null;
return docsEnum;
}
@Override
public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
if (fieldInfo.omitTermFreqAndPositions) {
return null;
} else {
return postingsReader.docsAndPositions(fieldInfo, state, skipDocs, reuse);
}
}
@Override
public SeekStatus seek(BytesRef term, TermState otherState) throws IOException {
assert otherState != null && otherState instanceof PrefixCodedTermState;
assert otherState.getClass() == this.state.getClass() : "Illegal TermState type " + otherState.getClass();
assert ((PrefixCodedTermState)otherState).ord < numTerms;
setTermState(term, otherState);
positioned = false;
return SeekStatus.FOUND;
}
@Override
public TermState termState() throws IOException {
final PrefixCodedTermState newTermState = (PrefixCodedTermState) state.clone();
newTermState.filePointer = in.getFilePointer();
return newTermState;
}
@Override
public SeekStatus seek(long ord) throws IOException {
if (indexEnum == null) {
throw new IllegalStateException("terms index was not loaded");
}
if (ord >= numTerms) {
state.ord = numTerms-1;
return SeekStatus.END;
}
in.seek(indexEnum.seek(ord));
seekPending = false;
positioned = true;
// NOTE: the first next() after an index seek is
// wasteful, since it redundantly reads the same
// bytes into the buffer
bytesReader.reset(indexEnum.term());
state.ord = indexEnum.ord()-1;
assert state.ord >= -1: "ord=" + state.ord;
// Now, scan:
int left = (int) (ord - state.ord);
while(left > 0) {
final BytesRef term = next();
assert term != null;
left--;
}
// always found
return SeekStatus.FOUND;
}
public long ord() {
if (!doOrd) {
throw new UnsupportedOperationException();
}
return state.ord;
}
}
}
}

View File

@ -17,7 +17,6 @@ package org.apache.lucene.index.codecs;
* limitations under the License.
*/
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
@ -25,14 +24,13 @@ import java.io.IOException;
/** @lucene.experimental */
public abstract class TermsIndexWriterBase {
public abstract void setTermsOutput(IndexOutput out);
public abstract class FieldWriter {
public abstract boolean checkIndexTerm(BytesRef text, TermStats stats) throws IOException;
public abstract void finish() throws IOException;
public abstract void add(BytesRef text, TermStats stats, long termsFilePointer) throws IOException;
public abstract void finish(long termsFilePointer) throws IOException;
}
public abstract FieldWriter addField(FieldInfo fieldInfo) throws IOException;
public abstract FieldWriter addField(FieldInfo fieldInfo, long termsFilePointer) throws IOException;
public abstract void close() throws IOException;
}

View File

@ -164,9 +164,6 @@ public class VariableGapTermsIndexReader extends TermsIndexReaderBase {
this.fieldInfo = fieldInfo;
this.indexStart = indexStart;
// We still create the indexReader when indexDivisor
// is -1, so that PrefixCodedTermsReader can call
// isIndexTerm for each field:
if (indexDivisor > 0) {
loadTermsIndex();
}

View File

@ -52,7 +52,6 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
private final List<FSTFieldWriter> fields = new ArrayList<FSTFieldWriter>();
private final FieldInfos fieldInfos; // unread
private IndexOutput termsOut;
private final IndexTermSelector policy;
/** @lucene.experimental */
@ -60,6 +59,7 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
// Called sequentially on every term being written,
// returning true if this term should be indexed
public abstract boolean isIndexTerm(BytesRef term, TermStats stats);
public abstract void newField(FieldInfo fieldInfo);
}
/** Same policy as {@link FixedGapTermsIndexWriter} */
@ -83,6 +83,11 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
return false;
}
}
@Override
public void newField(FieldInfo fieldInfo) {
count = interval;
}
}
/** Sets an index term when docFreq >= docFreqThresh, or
@ -96,6 +101,9 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
public EveryNOrDocFreqTermSelector(int docFreqThresh, int interval) {
this.interval = interval;
this.docFreqThresh = docFreqThresh;
// First term is first indexed term:
count = interval;
}
@Override
@ -108,6 +116,11 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
return false;
}
}
@Override
public void newField(FieldInfo fieldInfo) {
count = interval;
}
}
// TODO: it'd be nice to let the FST builder prune based
@ -158,14 +171,10 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
}
@Override
public void setTermsOutput(IndexOutput termsOut) {
this.termsOut = termsOut;
}
@Override
public FieldWriter addField(FieldInfo field) throws IOException {
//System.out.println("VGW: field=" + field.name);
FSTFieldWriter writer = new FSTFieldWriter(field);
public FieldWriter addField(FieldInfo field, long termsFilePointer) throws IOException {
////System.out.println("VGW: field=" + field.name);
policy.newField(field);
FSTFieldWriter writer = new FSTFieldWriter(field, termsFilePointer);
fields.add(writer);
return writer;
}
@ -200,42 +209,48 @@ public class VariableGapTermsIndexWriter extends TermsIndexWriterBase {
private final BytesRef lastTerm = new BytesRef();
private boolean first = true;
public FSTFieldWriter(FieldInfo fieldInfo) throws IOException {
public FSTFieldWriter(FieldInfo fieldInfo, long termsFilePointer) throws IOException {
this.fieldInfo = fieldInfo;
fstOutputs = PositiveIntOutputs.getSingleton(true);
fstBuilder = new Builder<Long>(FST.INPUT_TYPE.BYTE1,
0, 0, true,
fstOutputs);
indexStart = out.getFilePointer();
//System.out.println("VGW: field=" + fieldInfo.name);
////System.out.println("VGW: field=" + fieldInfo.name);
// Always put empty string in
fstBuilder.add(new BytesRef(), fstOutputs.get(termsOut.getFilePointer()));
fstBuilder.add(new BytesRef(), fstOutputs.get(termsFilePointer));
}
@Override
public boolean checkIndexTerm(BytesRef text, TermStats stats) throws IOException {
//System.out.println("VGW: index term=" + text.utf8ToString());
// NOTE: we must force the first term per field to be
// indexed, in case policy doesn't:
if (policy.isIndexTerm(text, stats) || first) {
first = false;
//System.out.println("VGW: index term=" + text.utf8ToString() + " fp=" + termsOut.getFilePointer());
final int lengthSave = text.length;
text.length = indexedTermPrefixLength(lastTerm, text);
try {
fstBuilder.add(text, fstOutputs.get(termsOut.getFilePointer()));
} finally {
text.length = lengthSave;
}
lastTerm.copy(text);
//System.out.println(" YES");
return true;
} else {
//System.out.println("VGW: not index term=" + text.utf8ToString() + " fp=" + termsOut.getFilePointer());
lastTerm.copy(text);
return false;
}
}
@Override
public void finish() throws IOException {
public void add(BytesRef text, TermStats stats, long termsFilePointer) throws IOException {
final int lengthSave = text.length;
text.length = indexedTermPrefixLength(lastTerm, text);
try {
fstBuilder.add(text, fstOutputs.get(termsFilePointer));
} finally {
text.length = lengthSave;
}
lastTerm.copy(text);
}
@Override
public void finish(long termsFilePointer) throws IOException {
fst = fstBuilder.finish();
if (fst != null) {
fst.save(out);

View File

@ -24,6 +24,7 @@ package org.apache.lucene.index.codecs.intblock;
import java.io.IOException;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IntsRef;
@ -149,7 +150,7 @@ public abstract class FixedIntBlockIndexInput extends IntIndexInput {
private int upto;
@Override
public void read(final IndexInput indexIn, final boolean absolute) throws IOException {
public void read(final DataInput indexIn, final boolean absolute) throws IOException {
if (absolute) {
fp = indexIn.readVLong();
upto = indexIn.readVInt();
@ -205,5 +206,10 @@ public abstract class FixedIntBlockIndexInput extends IntIndexInput {
other.upto = upto;
return other;
}
@Override
public String toString() {
return "fp=" + fp + " upto=" + upto;
}
}
}

View File

@ -111,6 +111,11 @@ public abstract class FixedIntBlockIndexOutput extends IntIndexOutput {
lastUpto = upto;
lastFP = fp;
}
@Override
public String toString() {
return "fp=" + fp + " upto=" + upto;
}
}
@Override

View File

@ -24,6 +24,7 @@ package org.apache.lucene.index.codecs.intblock;
import java.io.IOException;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IntsRef;
@ -168,7 +169,7 @@ public abstract class VariableIntBlockIndexInput extends IntIndexInput {
private int upto;
@Override
public void read(final IndexInput indexIn, final boolean absolute) throws IOException {
public void read(final DataInput indexIn, final boolean absolute) throws IOException {
if (absolute) {
fp = indexIn.readVLong();
upto = indexIn.readByte()&0xFF;

View File

@ -32,8 +32,8 @@ import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.standard.StandardCodec;
@ -89,7 +89,7 @@ public class PulsingCodec extends Codec {
// Terms dict
success = false;
try {
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, pulsingWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
success = true;
return ret;
} finally {
@ -132,13 +132,13 @@ public class PulsingCodec extends Codec {
// Terms dict reader
success = false;
try {
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
state.dir, state.fieldInfos, state.segmentInfo.name,
pulsingReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir, state.fieldInfos, state.segmentInfo.name,
pulsingReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
} finally {
@ -155,7 +155,7 @@ public class PulsingCodec extends Codec {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
StandardPostingsReader.files(dir, segmentInfo, id, files);
PrefixCodedTermsReader.files(dir, segmentInfo, id, files);
BlockTermsReader.files(dir, segmentInfo, id, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, id, files);
}

View File

@ -24,7 +24,7 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.PrefixCodedTermState;
import org.apache.lucene.index.codecs.BlockTermState;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.ArrayUtil;
@ -58,11 +58,13 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
wrappedPostingsReader.init(termsIn);
}
private static class PulsingTermState extends PrefixCodedTermState {
private static class PulsingTermState extends BlockTermState {
private byte[] postings;
private int postingsSize; // -1 if this term was not inlined
private PrefixCodedTermState wrappedTermState;
private boolean pendingIndexTerm;
private BlockTermState wrappedTermState;
ByteArrayDataInput inlinedBytesReader;
private byte[] inlinedBytes;
@Override
public Object clone() {
@ -73,7 +75,7 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
System.arraycopy(postings, 0, clone.postings, 0, postingsSize);
} else {
assert wrappedTermState != null;
clone.wrappedTermState = (PrefixCodedTermState) wrappedTermState.clone();
clone.wrappedTermState = (BlockTermState) wrappedTermState.clone();
}
return clone;
}
@ -91,56 +93,86 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
} else {
wrappedTermState.copyFrom(other.wrappedTermState);
}
// NOTE: we do not copy the
// inlinedBytes/inlinedBytesReader; these are only
// stored on the "primary" TermState. They are
// "transient" to cloned term states.
}
@Override
public String toString() {
if (postingsSize == -1) {
return "PulsingTermState: not inlined";
return "PulsingTermState: not inlined: wrapped=" + wrappedTermState;
} else {
return "PulsingTermState: inlined size=" + postingsSize;
return "PulsingTermState: inlined size=" + postingsSize + " " + super.toString();
}
}
}
@Override
public PrefixCodedTermState newTermState() throws IOException {
public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
final PulsingTermState termState = (PulsingTermState) _termState;
if (termState.inlinedBytes == null) {
termState.inlinedBytes = new byte[128];
termState.inlinedBytesReader = new ByteArrayDataInput(null);
}
int len = termsIn.readVInt();
if (termState.inlinedBytes.length < len) {
termState.inlinedBytes = new byte[ArrayUtil.oversize(len, 1)];
}
termsIn.readBytes(termState.inlinedBytes, 0, len);
termState.inlinedBytesReader.reset(termState.inlinedBytes);
termState.wrappedTermState.termCount = 0;
wrappedPostingsReader.readTermsBlock(termsIn, fieldInfo, termState.wrappedTermState);
}
@Override
public BlockTermState newTermState() throws IOException {
PulsingTermState state = new PulsingTermState();
state.wrappedTermState = wrappedPostingsReader.newTermState();
return state;
}
@Override
public void readTerm(IndexInput termsIn, FieldInfo fieldInfo, PrefixCodedTermState _termState, boolean isIndexTerm) throws IOException {
public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
//System.out.println("PR nextTerm");
PulsingTermState termState = (PulsingTermState) _termState;
termState.pendingIndexTerm |= isIndexTerm;
// total TF, but in the omitTFAP case its computed based on docFreq.
long count = fieldInfo.omitTermFreqAndPositions ? termState.docFreq : termState.totalTermFreq;
//System.out.println(" count=" + count + " threshold=" + maxPositions);
if (count <= maxPositions) {
//System.out.println(" inlined");
// Inlined into terms dict -- just read the byte[] blob in,
// but don't decode it now (we only decode when a DocsEnum
// or D&PEnum is pulled):
termState.postingsSize = termsIn.readVInt();
termState.postingsSize = termState.inlinedBytesReader.readVInt();
if (termState.postings == null || termState.postings.length < termState.postingsSize) {
termState.postings = new byte[ArrayUtil.oversize(termState.postingsSize, 1)];
}
termsIn.readBytes(termState.postings, 0, termState.postingsSize);
// TODO: sort of silly to copy from one big byte[]
// (the blob holding all inlined terms' blobs for
// current term block) into another byte[] (just the
// blob for this term)...
termState.inlinedBytesReader.readBytes(termState.postings, 0, termState.postingsSize);
} else {
//System.out.println(" not inlined");
termState.postingsSize = -1;
// TODO: should we do full copyFrom? much heavier...?
termState.wrappedTermState.docFreq = termState.docFreq;
wrappedPostingsReader.readTerm(termsIn, fieldInfo, termState.wrappedTermState, termState.pendingIndexTerm);
termState.pendingIndexTerm = false;
termState.wrappedTermState.totalTermFreq = termState.totalTermFreq;
wrappedPostingsReader.nextTerm(fieldInfo, termState.wrappedTermState);
termState.wrappedTermState.termCount++;
}
}
// TODO: we could actually reuse, by having TL that
// holds the last wrapped reuse, and vice-versa
@Override
public DocsEnum docs(FieldInfo field, PrefixCodedTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
public DocsEnum docs(FieldInfo field, BlockTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
PulsingTermState termState = (PulsingTermState) _termState;
if (termState.postingsSize != -1) {
PulsingDocsEnum postings;
@ -165,7 +197,7 @@ public class PulsingPostingsReaderImpl extends PostingsReaderBase {
// TODO: -- not great that we can't always reuse
@Override
public DocsAndPositionsEnum docsAndPositions(FieldInfo field, PrefixCodedTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
public DocsAndPositionsEnum docsAndPositions(FieldInfo field, BlockTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
if (field.omitTermFreqAndPositions) {
return null;
}

View File

@ -27,8 +27,8 @@ import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
// TODO: we pulse based on total TF of the term,
// it might be better to eg pulse by "net bytes used"
// TODO: we now inline based on total TF of the term,
// but it might be better to inline by "net bytes used"
// so that a term that has only 1 posting but a huge
// payload would not be inlined. Though this is
// presumably rare in practice...
@ -62,8 +62,9 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
}
// TODO: -- lazy init this? ie, if every single term
// was pulsed then we never need to use this fallback?
// Fallback writer for non-pulsed terms:
// was inlined (eg for a "primary key" field) then we
// never need to use this fallback? Fallback writer for
// non-inlined terms:
final PostingsWriterBase wrappedPostingsWriter;
/** If the total number of positions (summed across all docs
@ -173,22 +174,18 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
}
}
private boolean pendingIsIndexTerm;
private final RAMOutputStream buffer = new RAMOutputStream();
private final RAMOutputStream buffer2 = new RAMOutputStream();
/** Called when we are done adding docs to this term */
@Override
public void finishTerm(TermStats stats, boolean isIndexTerm) throws IOException {
//System.out.println("PW finishTerm docCount=" + docCount);
public void finishTerm(TermStats stats) throws IOException {
//System.out.println("PW finishTerm docCount=" + stats.docFreq);
assert pendingCount > 0 || pendingCount == -1;
pendingIsIndexTerm |= isIndexTerm;
if (pendingCount == -1) {
wrappedPostingsWriter.finishTerm(stats, pendingIsIndexTerm);
pendingIsIndexTerm = false;
wrappedPostingsWriter.finishTerm(stats);
} else {
// There were few enough total occurrences for this
@ -254,8 +251,8 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
}
//System.out.println(" bytes=" + buffer.getFilePointer());
termsOut.writeVInt((int) buffer.getFilePointer());
buffer.writeTo(termsOut);
buffer2.writeVInt((int) buffer.getFilePointer());
buffer.writeTo(buffer2);
buffer.reset();
}
@ -267,6 +264,18 @@ public final class PulsingPostingsWriterImpl extends PostingsWriterBase {
wrappedPostingsWriter.close();
}
@Override
public void flushTermsBlock() throws IOException {
termsOut.writeVInt((int) buffer2.getFilePointer());
buffer2.writeTo(termsOut);
buffer2.reset();
// TODO: can we avoid calling this if all terms
// were inlined...? Eg for a "primary key" field, the
// wrapped codec is never invoked...
wrappedPostingsWriter.flushTermsBlock();
}
// Pushes pending positions to the wrapped codec
private void push() throws IOException {
//System.out.println("PW now push @ " + pendingCount + " wrapped=" + wrappedPostingsWriter);

View File

@ -17,11 +17,11 @@ package org.apache.lucene.index.codecs.sep;
* limitations under the License.
*/
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IntsRef;
import java.io.IOException;
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.util.IntsRef;
/** Defines basic API for writing ints to an IndexOutput.
* IntBlockCodec interacts with this API. @see
@ -39,7 +39,7 @@ public abstract class IntIndexInput implements Closeable {
// TODO: -- can we simplify this?
public abstract static class Index {
public abstract void read(IndexInput indexIn, boolean absolute) throws IOException;
public abstract void read(DataInput indexIn, boolean absolute) throws IOException;
public abstract void read(IntIndexInput.Reader indexIn, boolean absolute) throws IOException;

View File

@ -20,16 +20,18 @@ package org.apache.lucene.index.codecs.sep;
import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.codecs.BlockTermState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.PrefixCodedTermState;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
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.CodecUtil;
@ -130,16 +132,31 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
}
}
private static final class SepTermState extends PrefixCodedTermState {
private static final class SepTermState extends BlockTermState {
// We store only the seek point to the docs file because
// the rest of the info (freqIndex, posIndex, etc.) is
// stored in the docs file:
IntIndexInput.Index docIndex;
IntIndexInput.Index posIndex;
IntIndexInput.Index freqIndex;
long payloadFP;
long skipFP;
// Only used for "primary" term state; these are never
// copied on clone:
byte[] bytes;
ByteArrayDataInput bytesReader;
@Override
public Object clone() {
SepTermState other = (SepTermState) super.clone();
other.docIndex = (IntIndexInput.Index) docIndex.clone();
if (freqIndex != null) {
other.freqIndex = (IntIndexInput.Index) freqIndex.clone();
}
if (posIndex != null) {
other.posIndex = (IntIndexInput.Index) posIndex.clone();
}
return other;
}
@ -147,28 +164,87 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
super.copyFrom(_other);
SepTermState other = (SepTermState) _other;
docIndex.set(other.docIndex);
if (freqIndex != null && other.freqIndex != null) {
freqIndex.set(other.freqIndex);
}
if (posIndex != null && other.posIndex != null) {
posIndex.set(other.posIndex);
}
payloadFP = other.payloadFP;
skipFP = other.skipFP;
}
@Override
public String toString() {
return "tis.fp=" + filePointer + " docFreq=" + docFreq + " ord=" + ord + " docIndex=" + docIndex;
return super.toString() + " docIndex=" + docIndex + " freqIndex=" + freqIndex + " posIndex=" + posIndex + " payloadFP=" + payloadFP + " skipFP=" + skipFP;
}
}
@Override
public PrefixCodedTermState newTermState() throws IOException {
final SepTermState state = new SepTermState();
public BlockTermState newTermState() throws IOException {
final SepTermState state = new SepTermState();
state.docIndex = docIn.index();
if (freqIn != null) {
state.freqIndex = freqIn.index();
}
if (posIn != null) {
state.posIndex = posIn.index();
}
return state;
}
@Override
public void readTerm(IndexInput termsIn, FieldInfo fieldInfo, PrefixCodedTermState termState, boolean isIndexTerm) throws IOException {
((SepTermState) termState).docIndex.read(termsIn, isIndexTerm);
public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
final SepTermState termState = (SepTermState) _termState;
final int len = termsIn.readVInt();
//System.out.println("SepR.readTermsBlock len=" + len);
if (termState.bytes == null) {
termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
termState.bytesReader = new ByteArrayDataInput(termState.bytes);
} else if (termState.bytes.length < len) {
termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
}
termState.bytesReader.reset(termState.bytes, 0, len);
termsIn.readBytes(termState.bytes, 0, len);
}
@Override
public DocsEnum docs(FieldInfo fieldInfo, PrefixCodedTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
final SepTermState termState = (SepTermState) _termState;
//System.out.println("SepR.nextTerm termCount=" + termState.termCount);
//System.out.println(" docFreq=" + termState.docFreq);
final boolean isFirstTerm = termState.termCount == 0;
termState.docIndex.read(termState.bytesReader, isFirstTerm);
//System.out.println(" docIndex=" + termState.docIndex);
if (!fieldInfo.omitTermFreqAndPositions) {
termState.freqIndex.read(termState.bytesReader, isFirstTerm);
//System.out.println(" freqIndex=" + termState.freqIndex);
termState.posIndex.read(termState.bytesReader, isFirstTerm);
//System.out.println(" posIndex=" + termState.posIndex);
if (fieldInfo.storePayloads) {
if (isFirstTerm) {
termState.payloadFP = termState.bytesReader.readVLong();
} else {
termState.payloadFP += termState.bytesReader.readVLong();
}
//System.out.println(" payloadFP=" + termState.payloadFP);
}
}
if (termState.docFreq >= skipInterval) {
//System.out.println(" readSkip @ " + termState.bytesReader.pos);
if (isFirstTerm) {
termState.skipFP = termState.bytesReader.readVLong();
} else {
termState.skipFP += termState.bytesReader.readVLong();
}
//System.out.println(" skipFP=" + termState.skipFP);
} else if (isFirstTerm) {
termState.skipFP = termState.bytesReader.readVLong();
}
}
@Override
public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
final SepTermState termState = (SepTermState) _termState;
SepDocsEnum docsEnum;
if (reuse == null || !(reuse instanceof SepDocsEnum)) {
@ -187,7 +263,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
}
@Override
public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, PrefixCodedTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
assert !fieldInfo.omitTermFreqAndPositions;
final SepTermState termState = (SepTermState) _termState;
SepDocsAndPositionsEnum postingsEnum;
@ -219,7 +295,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
private Bits skipDocs;
private final IntIndexInput.Reader docReader;
private final IntIndexInput.Reader freqReader;
private long skipOffset;
private long skipFP;
private final IntIndexInput.Index docIndex;
private final IntIndexInput.Index freqIndex;
@ -260,18 +336,15 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
docIndex.seek(docReader);
if (!omitTF) {
freqIndex.read(docReader, true);
freqIndex.set(termState.freqIndex);
freqIndex.seek(freqReader);
posIndex.read(docReader, true);
// skip payload offset
docReader.readVLong();
} else {
freq = 1;
}
skipOffset = docReader.readVLong();
docFreq = termState.docFreq;
// NOTE: unused if docFreq < skipInterval:
skipFP = termState.skipFP;
count = 0;
doc = 0;
skipped = false;
@ -290,9 +363,11 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
count++;
// Decode next doc
//System.out.println("decode docDelta:");
doc += docReader.next();
if (!omitTF) {
//System.out.println("decode freq:");
freq = freqReader.next();
}
@ -300,13 +375,13 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
break;
}
}
return doc;
}
@Override
public int read() throws IOException {
// TODO: -- switch to bulk read api in IntIndexInput
//System.out.println("sepdocs read");
final int[] docs = bulkResult.docs.ints;
final int[] freqs = bulkResult.freqs.ints;
int i = 0;
@ -314,14 +389,17 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
while (i < length && count < docFreq) {
count++;
// manually inlined call to next() for speed
//System.out.println("decode doc");
doc += docReader.next();
if (!omitTF) {
//System.out.println("decode freq");
freq = freqReader.next();
}
if (skipDocs == null || !skipDocs.get(doc)) {
docs[i] = doc;
freqs[i] = freq;
//System.out.println(" docs[" + i + "]=" + doc + " count=" + count + " dF=" + docFreq);
i++;
}
}
@ -361,7 +439,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
if (!skipped) {
// We haven't yet skipped for this posting
skipper.init(skipOffset,
skipper.init(skipFP,
docIndex,
freqIndex,
posIndex,
@ -411,14 +489,14 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
private final IntIndexInput.Reader freqReader;
private final IntIndexInput.Reader posReader;
private final IndexInput payloadIn;
private long skipOffset;
private long skipFP;
private final IntIndexInput.Index docIndex;
private final IntIndexInput.Index freqIndex;
private final IntIndexInput.Index posIndex;
private final IntIndexInput startDocIn;
private long payloadOffset;
private long payloadFP;
private int pendingPosCount;
private int position;
@ -444,21 +522,26 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits skipDocs) throws IOException {
this.skipDocs = skipDocs;
storePayloads = fieldInfo.storePayloads;
//System.out.println("Sep D&P init");
// TODO: can't we only do this if consumer
// skipped consuming the previous docs?
docIndex.set(termState.docIndex);
docIndex.seek(docReader);
//System.out.println(" docIndex=" + docIndex);
freqIndex.read(docReader, true);
freqIndex.set(termState.freqIndex);
freqIndex.seek(freqReader);
//System.out.println(" freqIndex=" + freqIndex);
posIndex.read(docReader, true);
posIndex.set(termState.posIndex);
//System.out.println(" posIndex=" + posIndex);
posSeekPending = true;
payloadPending = false;
payloadOffset = docReader.readVLong();
skipOffset = docReader.readVLong();
payloadFP = termState.payloadFP;
skipFP = termState.skipFP;
//System.out.println(" skipFP=" + skipFP);
docFreq = termState.docFreq;
count = 0;
@ -484,8 +567,10 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
// freq=1 case?
// Decode next doc
//System.out.println(" sep d&p read doc");
doc += docReader.next();
//System.out.println(" sep d&p read freq");
freq = freqReader.next();
pendingPosCount += freq;
@ -511,6 +596,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
@Override
public int advance(int target) throws IOException {
//System.out.println("SepD&P advance target=" + target + " vs current=" + doc + " this=" + this);
// TODO: jump right to next() if target is < X away
// from where we are now?
@ -521,6 +607,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
// skip data
if (skipper == null) {
//System.out.println(" create skipper");
// This DocsEnum has never done any skipping
skipper = new SepSkipListReader((IndexInput) skipIn.clone(),
freqIn,
@ -530,46 +617,54 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
}
if (!skipped) {
//System.out.println(" init skip data skipFP=" + skipFP);
// We haven't yet skipped for this posting
skipper.init(skipOffset,
skipper.init(skipFP,
docIndex,
freqIndex,
posIndex,
payloadOffset,
payloadFP,
docFreq,
storePayloads);
skipped = true;
}
final int newCount = skipper.skipTo(target);
//System.out.println(" skip newCount=" + newCount + " vs " + count);
if (newCount > count) {
// Skipper did move
skipper.getFreqIndex().seek(freqReader);
skipper.getDocIndex().seek(docReader);
//skipper.getPosIndex().seek(posReader);
// NOTE: don't seek pos here; do it lazily
// instead. Eg a PhraseQuery may skip to many
// docs before finally asking for positions...
posIndex.set(skipper.getPosIndex());
posSeekPending = true;
count = newCount;
doc = skipper.getDoc();
//System.out.println(" moved to doc=" + doc);
//payloadIn.seek(skipper.getPayloadPointer());
payloadOffset = skipper.getPayloadPointer();
payloadFP = skipper.getPayloadPointer();
pendingPosCount = 0;
pendingPayloadBytes = 0;
payloadPending = false;
payloadLength = skipper.getPayloadLength();
//System.out.println(" move payloadLen=" + payloadLength);
}
}
// Now, linear scan for the rest:
do {
if (nextDoc() == NO_MORE_DOCS) {
//System.out.println(" advance nextDoc=END");
return NO_MORE_DOCS;
}
//System.out.println(" advance nextDoc=" + doc);
} while (target > doc);
//System.out.println(" return doc=" + doc);
return doc;
}
@ -577,7 +672,7 @@ public class SepPostingsReaderImpl extends PostingsReaderBase {
public int nextPosition() throws IOException {
if (posSeekPending) {
posIndex.seek(posReader);
payloadIn.seek(payloadOffset);
payloadIn.seek(payloadFP);
posSeekPending = false;
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.TermStats;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
@ -69,8 +70,7 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
boolean storePayloads;
boolean omitTF;
// Starts a new term
long lastSkipStart;
long lastSkipFP;
FieldInfo fieldInfo;
@ -80,7 +80,10 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
long lastPayloadStart;
int lastDocID;
int df;
private boolean firstDoc;
private int pendingTermCount;
// Holds pending byte[] blob for the current terms block
private final RAMOutputStream indexBytesWriter = new RAMOutputStream();
public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory) throws IOException {
super();
@ -144,13 +147,9 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
payloadStart = payloadOut.getFilePointer();
lastPayloadLength = -1;
}
firstDoc = true;
skipListWriter.resetSkip(docIndex, freqIndex, posIndex);
}
// TODO: -- should we NOT reuse across fields? would
// be cleaner
// Currently, this instance is re-used across fields, so
// our parent calls setField whenever the field changes
@Override
@ -161,27 +160,13 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
storePayloads = !omitTF && fieldInfo.storePayloads;
}
/** Adds a new doc in this term. If this returns null
* then we just skip consuming positions/payloads. */
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
if (firstDoc) {
// TODO: we are writing absolute file pointers below,
// which is wasteful. It'd be better compression to
// write the "baseline" into each indexed term, then
// write only the delta here.
if (!omitTF) {
freqIndex.write(docOut, true);
posIndex.write(docOut, true);
docOut.writeVLong(payloadStart);
}
docOut.writeVLong(skipOut.getFilePointer());
firstDoc = false;
}
final int delta = docID - lastDocID;
//System.out.println("SepW startDoc: write doc=" + docID + " delta=" + delta);
if (docID < 0 || (df > 0 && delta <= 0)) {
throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
@ -190,6 +175,7 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
if ((++df % skipInterval) == 0) {
// TODO: -- awkward we have to make these two
// separate calls to skipper
//System.out.println(" buffer skip lastDocID=" + lastDocID);
skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
skipListWriter.bufferSkip(df);
}
@ -197,10 +183,20 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
lastDocID = docID;
docOut.write(delta);
if (!omitTF) {
//System.out.println(" sepw startDoc: write freq=" + termDocFreq);
freqOut.write(termDocFreq);
}
}
@Override
public void flushTermsBlock() throws IOException {
//System.out.println("SepW.flushTermsBlock: pendingTermCount=" + pendingTermCount + " bytesUsed=" + indexBytesWriter.getFilePointer());
termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
indexBytesWriter.writeTo(termsOut);
indexBytesWriter.reset();
pendingTermCount = 0;
}
/** Add a new position & payload */
@Override
public void addPosition(int position, BytesRef payload) throws IOException {
@ -240,20 +236,57 @@ public final class SepPostingsWriterImpl extends PostingsWriterBase {
/** Called when we are done adding docs to this term */
@Override
public void finishTerm(TermStats stats, boolean isIndexTerm) throws IOException {
public void finishTerm(TermStats stats) throws IOException {
// TODO: -- wasteful we are counting this in two places?
assert stats.docFreq > 0;
assert stats.docFreq == df;
docIndex.write(termsOut, isIndexTerm);
final boolean isFirstTerm = pendingTermCount == 0;
//System.out.println("SepW.finishTerm: isFirstTerm=" + isFirstTerm);
docIndex.write(indexBytesWriter, isFirstTerm);
//System.out.println(" docIndex=" + docIndex);
if (!omitTF) {
freqIndex.write(indexBytesWriter, isFirstTerm);
//System.out.println(" freqIndex=" + freqIndex);
posIndex.write(indexBytesWriter, isFirstTerm);
//System.out.println(" posIndex=" + posIndex);
if (storePayloads) {
if (isFirstTerm) {
indexBytesWriter.writeVLong(payloadStart);
} else {
indexBytesWriter.writeVLong(payloadStart - lastPayloadStart);
}
lastPayloadStart = payloadStart;
//System.out.println(" payloadFP=" + payloadStart);
}
}
if (df >= skipInterval) {
//System.out.println(" skipFP=" + skipStart);
final long skipFP = skipOut.getFilePointer();
skipListWriter.writeSkip(skipOut);
//System.out.println(" writeSkip @ " + indexBytesWriter.getFilePointer());
if (isFirstTerm) {
indexBytesWriter.writeVLong(skipFP);
} else {
indexBytesWriter.writeVLong(skipFP - lastSkipFP);
}
lastSkipFP = skipFP;
} else if (isFirstTerm) {
// TODO: this is somewhat wasteful; eg if no terms in
// this block will use skip data, we don't need to
// write this:
final long skipFP = skipOut.getFilePointer();
indexBytesWriter.writeVLong(skipFP);
lastSkipFP = skipFP;
}
lastDocID = 0;
df = 0;
pendingTermCount++;
}
@Override

View File

@ -33,8 +33,8 @@ import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.store.Directory;
/** Default codec.
@ -66,7 +66,7 @@ public class StandardCodec extends Codec {
success = false;
try {
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
success = true;
return ret;
} finally {
@ -103,15 +103,15 @@ public class StandardCodec extends Codec {
success = false;
try {
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postings,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
TERMS_CACHE_SIZE,
state.codecId);
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postings,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
} finally {
@ -134,7 +134,7 @@ public class StandardCodec extends Codec {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
StandardPostingsReader.files(dir, segmentInfo, id, files);
PrefixCodedTermsReader.files(dir, segmentInfo, id, files);
BlockTermsReader.files(dir, segmentInfo, id, files);
VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
}
@ -146,7 +146,7 @@ public class StandardCodec extends Codec {
public static void getStandardExtensions(Set<String> extensions) {
extensions.add(FREQ_EXTENSION);
extensions.add(PROX_EXTENSION);
PrefixCodedTermsReader.getExtensions(extensions);
BlockTermsReader.getExtensions(extensions);
VariableGapTermsIndexReader.getIndexExtensions(extensions);
}
}

View File

@ -20,16 +20,18 @@ package org.apache.lucene.index.codecs.standard;
import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.store.Directory;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.codecs.BlockTermState;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.PrefixCodedTermState;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.Directory;
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.CodecUtil;
@ -46,9 +48,12 @@ public class StandardPostingsReader extends PostingsReaderBase {
int skipInterval;
int maxSkipLevels;
//private String segment;
public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize, String codecId) throws IOException {
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
readBufferSize);
//this.segment = segmentInfo.name;
if (segmentInfo.getHasProx()) {
boolean success = false;
try {
@ -84,11 +89,16 @@ public class StandardPostingsReader extends PostingsReaderBase {
}
// Must keep final because we do non-standard clone
private final static class StandardTermState extends PrefixCodedTermState {
private final static class StandardTermState extends BlockTermState {
long freqOffset;
long proxOffset;
int skipOffset;
// Only used by the "primary" TermState -- clones don't
// copy this (basically they are "transient"):
ByteArrayDataInput bytesReader;
byte[] bytes;
public Object clone() {
StandardTermState other = new StandardTermState();
other.copyFrom(this);
@ -101,6 +111,11 @@ public class StandardPostingsReader extends PostingsReaderBase {
freqOffset = other.freqOffset;
proxOffset = other.proxOffset;
skipOffset = other.skipOffset;
// Do not copy bytes, bytesReader (else TermState is
// very heavy, ie drags around the entire block's
// byte[]). On seek back, if next() is in fact used
// (rare!), they will be re-read from disk.
}
public String toString() {
@ -109,7 +124,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
}
@Override
public PrefixCodedTermState newTermState() {
public BlockTermState newTermState() {
return new StandardTermState();
}
@ -126,34 +141,58 @@ public class StandardPostingsReader extends PostingsReaderBase {
}
}
/* Reads but does not decode the byte[] blob holding
metadata for the current terms block */
@Override
public void readTerm(IndexInput termsIn, FieldInfo fieldInfo, PrefixCodedTermState termState, boolean isIndexTerm)
throws IOException {
final StandardTermState docTermState = (StandardTermState) termState;
public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
final StandardTermState termState = (StandardTermState) _termState;
if (isIndexTerm) {
docTermState.freqOffset = termsIn.readVLong();
} else {
docTermState.freqOffset += termsIn.readVLong();
final int len = termsIn.readVInt();
//System.out.println("SPR.readTermsBlock termsIn.fp=" + termsIn.getFilePointer());
if (termState.bytes == null) {
termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
termState.bytesReader = new ByteArrayDataInput(null);
} else if (termState.bytes.length < len) {
termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
}
if (docTermState.docFreq >= skipInterval) {
docTermState.skipOffset = termsIn.readVInt();
termsIn.readBytes(termState.bytes, 0, len);
termState.bytesReader.reset(termState.bytes, 0, len);
}
@Override
public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
throws IOException {
final StandardTermState termState = (StandardTermState) _termState;
//System.out.println("StandardR.nextTerm seg=" + segment);
final boolean isFirstTerm = termState.termCount == 0;
if (isFirstTerm) {
termState.freqOffset = termState.bytesReader.readVLong();
} else {
docTermState.skipOffset = 0;
termState.freqOffset += termState.bytesReader.readVLong();
}
//System.out.println(" freqFP=" + termState.freqOffset);
if (termState.docFreq >= skipInterval) {
termState.skipOffset = termState.bytesReader.readVInt();
//System.out.println(" skipOffset=" + termState.skipOffset);
} else {
// undefined
}
if (!fieldInfo.omitTermFreqAndPositions) {
if (isIndexTerm) {
docTermState.proxOffset = termsIn.readVLong();
if (isFirstTerm) {
termState.proxOffset = termState.bytesReader.readVLong();
} else {
docTermState.proxOffset += termsIn.readVLong();
termState.proxOffset += termState.bytesReader.readVLong();
}
//System.out.println(" proxFP=" + termState.proxOffset);
}
}
@Override
public DocsEnum docs(FieldInfo fieldInfo, PrefixCodedTermState termState, Bits skipDocs, DocsEnum reuse) throws IOException {
public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits skipDocs, DocsEnum reuse) throws IOException {
SegmentDocsEnum docsEnum;
if (reuse == null || !(reuse instanceof SegmentDocsEnum)) {
docsEnum = new SegmentDocsEnum(freqIn);
@ -170,7 +209,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
}
@Override
public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, PrefixCodedTermState termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
if (fieldInfo.omitTermFreqAndPositions) {
return null;
}
@ -248,8 +287,10 @@ public class StandardPostingsReader extends PostingsReaderBase {
// cases
freqIn.seek(termState.freqOffset);
limit = termState.docFreq;
assert limit > 0;
ord = 0;
doc = 0;
//System.out.println(" sde limit=" + limit + " freqFP=" + freqOffset);
skipped = false;
@ -420,6 +461,8 @@ public class StandardPostingsReader extends PostingsReaderBase {
lazyProxPointer = termState.proxOffset;
limit = termState.docFreq;
assert limit > 0;
ord = 0;
doc = 0;
position = 0;
@ -430,6 +473,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
freqOffset = termState.freqOffset;
proxOffset = termState.proxOffset;
skipOffset = termState.skipOffset;
//System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
return this;
}
@ -438,6 +482,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
public int nextDoc() throws IOException {
while(true) {
if (ord == limit) {
//System.out.println("StandardR.D&PE seg=" + segment + " nextDoc return doc=END");
return doc = NO_MORE_DOCS;
}
@ -461,6 +506,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
position = 0;
//System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc);
return doc;
}
@ -477,6 +523,8 @@ public class StandardPostingsReader extends PostingsReaderBase {
@Override
public int advance(int target) throws IOException {
//System.out.println("StandardR.D&PE advance target=" + target);
// TODO: jump right to next() if target is < X away
// from where we are now?
@ -622,6 +670,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
freqOffset = termState.freqOffset;
proxOffset = termState.proxOffset;
skipOffset = termState.skipOffset;
//System.out.println("StandardR.D&PE reset seg=" + segment + " limit=" + limit + " freqFP=" + freqOffset + " proxFP=" + proxOffset);
return this;
}
@ -630,6 +679,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
public int nextDoc() throws IOException {
while(true) {
if (ord == limit) {
//System.out.println("StandardR.D&PE seg=" + segment + " nextDoc return doc=END");
return doc = NO_MORE_DOCS;
}
@ -653,6 +703,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
position = 0;
//System.out.println("StandardR.D&PE nextDoc seg=" + segment + " return doc=" + doc);
return doc;
}
@ -748,6 +799,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
posPendingCount--;
position = 0;
payloadPending = false;
//System.out.println("StandardR.D&PE skipPos");
}
// read next position
@ -771,6 +823,7 @@ public class StandardPostingsReader extends PostingsReaderBase {
assert posPendingCount >= 0: "nextPosition() was called too many times (more than freq() times) posPendingCount=" + posPendingCount;
//System.out.println("StandardR.D&PE nextPos return pos=" + position);
return position;
}

View File

@ -22,13 +22,14 @@ package org.apache.lucene.index.codecs.standard;
import java.io.IOException;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.TermStats;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CodecUtil;
@ -59,8 +60,15 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
int lastPayloadLength;
int lastPosition;
private int pendingCount;
//private String segment;
private RAMOutputStream bytesWriter = new RAMOutputStream();
public StandardPostingsWriter(SegmentWriteState state) throws IOException {
super();
//this.segment = state.segmentName;
String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
freqOut = state.directory.createOutput(fileName);
@ -96,6 +104,7 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
@Override
public void startTerm() {
//System.out.println("StandardW: startTerm seg=" + segment + " pendingCount=" + pendingCount);
freqStart = freqOut.getFilePointer();
if (proxOut != null) {
proxStart = proxOut.getFilePointer();
@ -109,9 +118,12 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
// our parent calls setField whenever the field changes
@Override
public void setField(FieldInfo fieldInfo) {
//System.out.println("SPW: setField");
this.fieldInfo = fieldInfo;
omitTermFreqAndPositions = fieldInfo.omitTermFreqAndPositions;
storePayloads = fieldInfo.storePayloads;
//System.out.println(" set init blockFreqStart=" + freqStart);
//System.out.println(" set init blockProxStart=" + proxStart);
}
int lastDocID;
@ -121,6 +133,7 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
* then we just skip consuming positions/payloads. */
@Override
public void startDoc(int docID, int termDocFreq) throws IOException {
//System.out.println("StandardW: startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq);
final int delta = docID - lastDocID;
@ -151,6 +164,7 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
/** Add a new position & payload */
@Override
public void addPosition(int position, BytesRef payload) throws IOException {
//System.out.println("StandardW: addPos pos=" + position + " payload=" + (payload == null ? "null" : (payload.length + " bytes")) + " proxFP=" + proxOut.getFilePointer());
assert !omitTermFreqAndPositions: "omitTermFreqAndPositions is true";
assert proxOut != null;
@ -185,40 +199,51 @@ public final class StandardPostingsWriter extends PostingsWriterBase {
/** Called when we are done adding docs to this term */
@Override
public void finishTerm(TermStats stats, boolean isIndexTerm) throws IOException {
public void finishTerm(TermStats stats) throws IOException {
//System.out.println("StandardW.finishTerm seg=" + segment);
assert stats.docFreq > 0;
// TODO: wasteful we are counting this (counting # docs
// for this term) in two places?
assert stats.docFreq == df;
if (isIndexTerm) {
// Write absolute at seek points
termsOut.writeVLong(freqStart);
} else {
// Write delta between seek points
termsOut.writeVLong(freqStart - lastFreqStart);
}
final boolean isFirstTerm = pendingCount == 0;
//System.out.println(" isFirstTerm=" + isFirstTerm);
//System.out.println(" freqFP=" + freqStart);
if (isFirstTerm) {
bytesWriter.writeVLong(freqStart);
} else {
bytesWriter.writeVLong(freqStart-lastFreqStart);
}
lastFreqStart = freqStart;
if (df >= skipInterval) {
termsOut.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
bytesWriter.writeVInt((int) (skipListWriter.writeSkip(freqOut)-freqStart));
}
if (!omitTermFreqAndPositions) {
if (isIndexTerm) {
// Write absolute at seek points
termsOut.writeVLong(proxStart);
//System.out.println(" proxFP=" + proxStart);
if (isFirstTerm) {
bytesWriter.writeVLong(proxStart);
} else {
// Write delta between seek points
termsOut.writeVLong(proxStart - lastProxStart);
bytesWriter.writeVLong(proxStart - lastProxStart);
}
lastProxStart = proxStart;
}
lastDocID = 0;
df = 0;
pendingCount++;
}
@Override
public void flushTermsBlock() throws IOException {
//System.out.println("SPW.flushBlock pendingCount=" + pendingCount);
termsOut.writeVInt((int) bytesWriter.getFilePointer());
bytesWriter.writeTo(termsOut);
bytesWriter.reset();
pendingCount = 0;
}
@Override

View File

@ -122,12 +122,12 @@ public abstract class FilteredTermsEnum extends TermsEnum {
}
@Override
public int docFreq() {
public int docFreq() throws IOException {
return tenum.docFreq();
}
@Override
public long totalTermFreq() {
public long totalTermFreq() throws IOException {
return tenum.totalTermFreq();
}
@ -166,7 +166,7 @@ public abstract class FilteredTermsEnum extends TermsEnum {
* @throws UnsupportedOperationException
*/
@Override
public SeekStatus seek(BytesRef term, TermState state) throws IOException {
public void seek(BytesRef term, TermState state) throws IOException {
throw new UnsupportedOperationException(getClass().getName()+" does not support seeking");
}

View File

@ -241,12 +241,12 @@ public final class FuzzyTermsEnum extends TermsEnum {
// proxy all other enum calls to the actual enum
@Override
public int docFreq() {
public int docFreq() throws IOException {
return actualEnum.docFreq();
}
@Override
public long totalTermFreq() {
public long totalTermFreq() throws IOException {
return actualEnum.totalTermFreq();
}
@ -261,8 +261,8 @@ public final class FuzzyTermsEnum extends TermsEnum {
return actualEnum.docsAndPositions(skipDocs, reuse);
}
public SeekStatus seek(BytesRef term, TermState state) throws IOException {
return actualEnum.seek(term, state);
public void seek(BytesRef term, TermState state) throws IOException {
actualEnum.seek(term, state);
}
@Override

View File

@ -341,9 +341,9 @@ public class DocTermsIndexCreator extends EntryCreatorWithOptions<DocTermsIndex>
}
@Override
public SeekStatus seek(BytesRef term, TermState state) throws IOException {
public void seek(BytesRef term, TermState state) throws IOException {
assert state != null && state instanceof OrdTermState;
return this.seek(((OrdTermState)state).ord);
this.seek(((OrdTermState)state).ord);
}
@Override

View File

@ -21,7 +21,9 @@ package org.apache.lucene.store;
public final class ByteArrayDataInput extends DataInput {
private byte[] bytes;
private int pos;
private int limit;
// TODO: allow BytesRef (slice) too
public ByteArrayDataInput(byte[] bytes) {
@ -29,27 +31,80 @@ public final class ByteArrayDataInput extends DataInput {
}
public void reset(byte[] bytes) {
reset(bytes, 0, bytes.length);
}
public int getPosition() {
return pos;
}
public void reset(byte[] bytes, int offset, int len) {
this.bytes = bytes;
pos = 0;
pos = offset;
limit = len;
}
public boolean eof() {
return pos == bytes.length;
return pos == limit;
}
public void skipBytes(int count) {
pos += count;
}
@Override
public short readShort() {
return (short) (((bytes[pos++] & 0xFF) << 8) | (bytes[pos++] & 0xFF));
}
@Override
public int readInt() {
return ((bytes[pos++] & 0xFF) << 24) | ((bytes[pos++] & 0xFF) << 16)
| ((bytes[pos++] & 0xFF) << 8) | (bytes[pos++] & 0xFF);
}
@Override
public long readLong() {
final int i1 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
final int i2 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
return (((long)i1) << 32) | (i2 & 0xFFFFFFFFL);
}
@Override
public int readVInt() {
byte b = bytes[pos++];
int i = b & 0x7F;
for (int shift = 7; (b & 0x80) != 0; shift += 7) {
b = bytes[pos++];
i |= (b & 0x7F) << shift;
}
return i;
}
@Override
public long readVLong() {
byte b = bytes[pos++];
long i = b & 0x7F;
for (int shift = 7; (b & 0x80) != 0; shift += 7) {
b = bytes[pos++];
i |= (b & 0x7FL) << shift;
}
return i;
}
// NOTE: AIOOBE not EOF if you read too much
@Override
public byte readByte() {
assert pos < limit;
return bytes[pos++];
}
// NOTE: AIOOBE not EOF if you read too much
@Override
public void readBytes(byte[] b, int offset, int len) {
assert pos + len <= limit;
System.arraycopy(bytes, pos, b, offset, len);
pos += len;
}

View File

@ -79,7 +79,7 @@ public abstract class DataInput implements Cloneable {
* supported.
* @see DataOutput#writeVInt(int)
*/
public final int readVInt() throws IOException {
public int readVInt() throws IOException {
byte b = readByte();
int i = b & 0x7F;
for (int shift = 7; (b & 0x80) != 0; shift += 7) {
@ -99,7 +99,7 @@ public abstract class DataInput implements Cloneable {
/** Reads a long stored in variable-length format. Reads between one and
* nine bytes. Smaller values take fewer bytes. Negative numbers are not
* supported. */
public final long readVLong() throws IOException {
public long readVLong() throws IOException {
byte b = readByte();
long i = b & 0x7F;
for (int shift = 7; (b & 0x80) != 0; shift += 7) {

View File

@ -245,7 +245,6 @@ public class TestExternalCodecs extends LuceneTestCase {
}
}
// Classes for reading from the postings state
static class RAMFieldsEnum extends FieldsEnum {
private final RAMPostings postings;
@ -543,7 +542,7 @@ public class TestExternalCodecs extends LuceneTestCase {
// Terms dict
success = false;
try {
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, pulsingWriter, reverseUnicodeComparator);
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter, reverseUnicodeComparator);
success = true;
return ret;
} finally {
@ -584,15 +583,15 @@ public class TestExternalCodecs extends LuceneTestCase {
// Terms dict reader
success = false;
try {
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
pulsingReader,
state.readBufferSize,
reverseUnicodeComparator,
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
pulsingReader,
state.readBufferSize,
reverseUnicodeComparator,
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
} finally {
@ -609,7 +608,7 @@ public class TestExternalCodecs extends LuceneTestCase {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) throws IOException {
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
}
@ -637,6 +636,7 @@ public class TestExternalCodecs extends LuceneTestCase {
setCodecProvider(provider).
setMergePolicy(newLogMergePolicy(3))
);
w.setInfoStream(VERBOSE ? System.out : null);
Document doc = new Document();
// uses default codec:
doc.add(newField("field1", "this field uses the standard codec as the test", Field.Store.NO, Field.Index.ANALYZED));

View File

@ -111,6 +111,9 @@ public class TestSearch extends LuceneTestCase {
for (int j = 0; j < queries.length; j++) {
Query query = parser.parse(queries[j]);
out.println("Query: " + query.toString("contents"));
if (VERBOSE) {
System.out.println("TEST: query=" + query);
}
hits = searcher.search(query, null, 1000).scoreDocs;

View File

@ -108,6 +108,9 @@ public class TestSearchForDuplicates extends LuceneTestCase {
Query query = parser.parse(HIGH_PRIORITY);
out.println("Query: " + query.toString(PRIORITY_FIELD));
if (VERBOSE) {
System.out.println("TEST: search query=" + query);
}
ScoreDoc[] hits = searcher.search(query, null, MAX_DOCS).scoreDocs;
printHits(out, hits, searcher);

View File

@ -55,6 +55,7 @@ public class TestAddIndexes extends LuceneTestCase {
writer = newWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer())
.setOpenMode(OpenMode.CREATE));
writer.setInfoStream(VERBOSE ? System.out : null);
// add 100 documents
addDocs(writer, 100);
assertEquals(100, writer.maxDoc());

View File

@ -882,6 +882,10 @@ public class TestIndexReader extends LuceneTestCase
// First build up a starting index:
MockDirectoryWrapper startDir = newDirectory();
IndexWriter writer = new IndexWriter(startDir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
if (VERBOSE) {
System.out.println("TEST: create initial index");
writer.setInfoStream(System.out);
}
for(int i=0;i<157;i++) {
Document d = new Document();
d.add(newField("id", Integer.toString(i), Field.Store.YES, Field.Index.NOT_ANALYZED));
@ -892,6 +896,19 @@ public class TestIndexReader extends LuceneTestCase
}
writer.close();
{
IndexReader r = IndexReader.open(startDir);
IndexSearcher searcher = new IndexSearcher(r);
ScoreDoc[] hits = null;
try {
hits = searcher.search(new TermQuery(searchTerm), null, 1000).scoreDocs;
} catch (IOException e) {
e.printStackTrace();
fail("exception when init searching: " + e);
}
r.close();
}
long diskUsage = startDir.getRecomputedActualSizeInBytes();
long diskFree = diskUsage+100;

View File

@ -977,7 +977,11 @@ public class TestIndexReaderReopen extends LuceneTestCase {
static void modifyIndex(int i, Directory dir) throws IOException {
switch (i) {
case 0: {
if (VERBOSE) {
System.out.println("TEST: modify index");
}
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
w.setInfoStream(VERBOSE ? System.out : null);
w.deleteDocuments(new Term("field2", "a11"));
w.deleteDocuments(new Term("field2", "b30"));
w.close();

View File

@ -97,6 +97,9 @@ public class TestMultiFields extends LuceneTestCase {
for(int i=0;i<100;i++) {
BytesRef term = terms.get(random.nextInt(terms.size()));
if (VERBOSE) {
System.out.println("TEST: seek to term= "+ UnicodeUtil.toHexString(term.utf8ToString()));
}
DocsEnum docsEnum = terms2.docs(delDocs, term, null);
assertNotNull(docsEnum);

View File

@ -37,8 +37,8 @@ import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.standard.StandardCodec;
@ -126,7 +126,7 @@ public class MockFixedIntBlockCodec extends Codec {
success = false;
try {
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
success = true;
return ret;
} finally {
@ -164,15 +164,15 @@ public class MockFixedIntBlockCodec extends Codec {
success = false;
try {
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
} finally {
@ -189,14 +189,14 @@ public class MockFixedIntBlockCodec extends Codec {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
}
@Override
public void getExtensions(Set<String> extensions) {
SepPostingsWriterImpl.getExtensions(extensions);
PrefixCodedTermsReader.getExtensions(extensions);
BlockTermsReader.getExtensions(extensions);
FixedGapTermsIndexReader.getIndexExtensions(extensions);
}
}

View File

@ -37,8 +37,8 @@ import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.standard.StandardCodec;
@ -150,7 +150,7 @@ public class MockVariableIntBlockCodec extends Codec {
success = false;
try {
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
success = true;
return ret;
} finally {
@ -189,15 +189,15 @@ public class MockVariableIntBlockCodec extends Codec {
success = false;
try {
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
} finally {
@ -214,14 +214,14 @@ public class MockVariableIntBlockCodec extends Codec {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
}
@Override
public void getExtensions(Set<String> extensions) {
SepPostingsWriterImpl.getExtensions(extensions);
PrefixCodedTermsReader.getExtensions(extensions);
BlockTermsReader.getExtensions(extensions);
FixedGapTermsIndexReader.getIndexExtensions(extensions);
}
}

View File

@ -26,6 +26,9 @@ import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
@ -33,13 +36,11 @@ import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
import org.apache.lucene.index.codecs.TermStats;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
import org.apache.lucene.index.codecs.TermStats;
import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec;
import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec;
import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory;
@ -152,6 +153,10 @@ public class MockRandomCodec extends Codec {
public boolean isIndexTerm(BytesRef term, TermStats stats) {
return random.nextInt(gap) == 17;
}
@Override
public void newField(FieldInfo fieldInfo) {
}
};
}
indexWriter = new VariableGapTermsIndexWriter(state, selector);
@ -165,7 +170,7 @@ public class MockRandomCodec extends Codec {
success = false;
try {
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
success = true;
return ret;
} finally {
@ -269,15 +274,15 @@ public class MockRandomCodec extends Codec {
success = false;
try {
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
termsCacheSize,
state.codecId);
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
termsCacheSize,
state.codecId);
success = true;
return ret;
} finally {
@ -297,7 +302,7 @@ public class MockRandomCodec extends Codec {
files.add(seedFileName);
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
StandardPostingsReader.files(dir, segmentInfo, codecId, files);
PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
@ -315,7 +320,7 @@ public class MockRandomCodec extends Codec {
@Override
public void getExtensions(Set<String> extensions) {
SepPostingsWriterImpl.getExtensions(extensions);
PrefixCodedTermsReader.getExtensions(extensions);
BlockTermsReader.getExtensions(extensions);
FixedGapTermsIndexReader.getIndexExtensions(extensions);
VariableGapTermsIndexReader.getIndexExtensions(extensions);
extensions.add(SEED_EXT);

View File

@ -30,8 +30,8 @@ import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
import org.apache.lucene.index.codecs.PostingsReaderBase;
import org.apache.lucene.index.codecs.PostingsWriterBase;
import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
import org.apache.lucene.index.codecs.BlockTermsReader;
import org.apache.lucene.index.codecs.BlockTermsWriter;
import org.apache.lucene.index.codecs.TermsIndexReaderBase;
import org.apache.lucene.index.codecs.TermsIndexWriterBase;
import org.apache.lucene.index.codecs.standard.StandardCodec;
@ -70,7 +70,7 @@ public class MockSepCodec extends Codec {
success = false;
try {
FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
success = true;
return ret;
} finally {
@ -108,15 +108,15 @@ public class MockSepCodec extends Codec {
success = false;
try {
FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
FieldsProducer ret = new BlockTermsReader(indexReader,
state.dir,
state.fieldInfos,
state.segmentInfo.name,
postingsReader,
state.readBufferSize,
BytesRef.getUTF8SortedAsUnicodeComparator(),
StandardCodec.TERMS_CACHE_SIZE,
state.codecId);
success = true;
return ret;
} finally {
@ -133,7 +133,7 @@ public class MockSepCodec extends Codec {
@Override
public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set<String> files) {
SepPostingsReaderImpl.files(segmentInfo, codecId, files);
PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files);
BlockTermsReader.files(dir, segmentInfo, codecId, files);
FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
}
@ -144,7 +144,7 @@ public class MockSepCodec extends Codec {
public static void getSepExtensions(Set<String> extensions) {
SepPostingsWriterImpl.getExtensions(extensions);
PrefixCodedTermsReader.getExtensions(extensions);
BlockTermsReader.getExtensions(extensions);
FixedGapTermsIndexReader.getIndexExtensions(extensions);
}
}
}

View File

@ -19,10 +19,11 @@ package org.apache.lucene.index.codecs.mocksep;
import java.io.IOException;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.CodecUtil;
import org.apache.lucene.index.codecs.sep.IntIndexInput;
/** Reads IndexInputs written with {@link
* SingleIntIndexOutput}. NOTE: this class is just for
@ -63,6 +64,7 @@ public class MockSingleIntIndexInput extends IntIndexInput {
/** Reads next single int */
@Override
public int next() throws IOException {
//System.out.println("msii.next() fp=" + in.getFilePointer() + " vs " + in.length());
return in.readVInt();
}
}
@ -71,7 +73,7 @@ public class MockSingleIntIndexInput extends IntIndexInput {
private long fp;
@Override
public void read(IndexInput indexIn, boolean absolute)
public void read(DataInput indexIn, boolean absolute)
throws IOException {
if (absolute) {
fp = indexIn.readVLong();

View File

@ -49,6 +49,10 @@ public class TestFieldCache extends LuceneTestCase {
int theInt = Integer.MAX_VALUE;
float theFloat = Float.MAX_VALUE;
unicodeStrings = new String[NUM_DOCS];
if (VERBOSE) {
System.out.println("TEST: setUp");
}
writer.w.setInfoStream(VERBOSE ? System.out : null);
for (int i = 0; i < NUM_DOCS; i++){
Document doc = new Document();
doc.add(newField("theLong", String.valueOf(theLong--), Field.Store.NO, Field.Index.NOT_ANALYZED));
@ -117,7 +121,7 @@ public class TestFieldCache extends LuceneTestCase {
assertSame("Second request with explicit parser return same array", longs, cache.getLongs(reader, "theLong", FieldCache.DEFAULT_LONG_PARSER));
assertTrue("longs Size: " + longs.length + " is not: " + NUM_DOCS, longs.length == NUM_DOCS);
for (int i = 0; i < longs.length; i++) {
assertTrue(longs[i] + " does not equal: " + (Long.MAX_VALUE - i), longs[i] == (Long.MAX_VALUE - i));
assertTrue(longs[i] + " does not equal: " + (Long.MAX_VALUE - i) + " i=" + i, longs[i] == (Long.MAX_VALUE - i));
}

View File

@ -40,7 +40,6 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.OrdTermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.CodecProvider;
@ -975,6 +974,9 @@ public class TestFSTs extends LuceneTestCase {
Terms terms = MultiFields.getTerms(r, "body");
if (terms != null) {
final TermsEnum termsEnum = terms.iterator();
if (VERBOSE) {
System.out.println("TEST: got termsEnum=" + termsEnum);
}
BytesRef term;
int ord = 0;
while((term = termsEnum.next()) != null) {
@ -982,6 +984,9 @@ public class TestFSTs extends LuceneTestCase {
try {
termsEnum.ord();
} catch (UnsupportedOperationException uoe) {
if (VERBOSE) {
System.out.println("TEST: codec doesn't support ord; FST stores docFreq");
}
storeOrd = false;
}
}
@ -1023,6 +1028,9 @@ public class TestFSTs extends LuceneTestCase {
for(int nextIter=0;nextIter<10;nextIter++) {
if (VERBOSE) {
System.out.println("TEST: next");
if (storeOrd) {
System.out.println(" ord=" + termsEnum.ord());
}
}
if (termsEnum.next() != null) {
if (VERBOSE) {

View File

@ -996,12 +996,12 @@ class NumberedTermsEnum extends TermsEnum {
}
@Override
public int docFreq() {
public int docFreq() throws IOException {
return tenum.docFreq();
}
@Override
public long totalTermFreq() {
public long totalTermFreq() throws IOException {
return tenum.totalTermFreq();
}

View File

@ -76,7 +76,11 @@ public class HighFrequencyDictionary implements Dictionary {
}
public float freq() {
return termsEnum.docFreq();
try {
return termsEnum.docFreq();
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
public String next() {
@ -112,8 +116,12 @@ public class HighFrequencyDictionary implements Dictionary {
}
// got a valid term, does it pass the threshold?
if (isFrequent(termsEnum.docFreq())) {
return true;
try {
if (isFrequent(termsEnum.docFreq())) {
return true;
}
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
}