mirror of https://github.com/apache/lucene.git
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:
parent
f320ade2b7
commit
1a0f78778b
|
@ -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
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
};
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -111,6 +111,11 @@ public abstract class FixedIntBlockIndexOutput extends IntIndexOutput {
|
|||
lastUpto = upto;
|
||||
lastFP = fp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "fp=" + fp + " upto=" + upto;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue