mirror of https://github.com/apache/lucene.git
LUCENE-3069: add TempFSTOrd, with FST index + specialized block
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3069@1508705 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
dc1bee7529
commit
f957c7648c
|
@ -0,0 +1,77 @@
|
|||
package org.apache.lucene.codecs.temp;
|
||||
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.TempPostingsReaderBase;
|
||||
import org.apache.lucene.codecs.TempPostingsWriterBase;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
public final class TempFSTOrdPostingsFormat extends PostingsFormat {
|
||||
public TempFSTOrdPostingsFormat() {
|
||||
super("TempFSTOrd");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getName();
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
|
||||
TempPostingsWriterBase postingsWriter = new TempPostingsWriter(state);
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
FieldsConsumer ret = new TempFSTOrdTermsWriter(state, postingsWriter);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(postingsWriter);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
|
||||
TempPostingsReaderBase postingsReader = new TempPostingsReader(state.directory,
|
||||
state.fieldInfos,
|
||||
state.segmentInfo,
|
||||
state.context,
|
||||
state.segmentSuffix);
|
||||
boolean success = false;
|
||||
try {
|
||||
FieldsProducer ret = new TempFSTOrdTermsReader(state, postingsReader);
|
||||
success = true;
|
||||
return ret;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(postingsReader);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,522 @@
|
|||
package org.apache.lucene.codecs.temp;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.io.File;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.index.TermState;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.automaton.ByteRunAutomaton;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.Outputs;
|
||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.TempPostingsReaderBase;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
|
||||
public class TempFSTOrdTermsReader extends FieldsProducer {
|
||||
static final int INTERVAL = TempFSTOrdTermsWriter.SKIP_INTERVAL;
|
||||
final TreeMap<String, TermsReader> fields = new TreeMap<String, TermsReader>();
|
||||
final TempPostingsReaderBase postingsReader;
|
||||
IndexInput indexIn = null;
|
||||
IndexInput blockIn = null;
|
||||
static final boolean TEST = false;
|
||||
|
||||
public TempFSTOrdTermsReader(SegmentReadState state, TempPostingsReaderBase postingsReader) throws IOException {
|
||||
final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempFSTOrdTermsWriter.TERMS_INDEX_EXTENSION);
|
||||
final String termsBlockFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TempFSTOrdTermsWriter.TERMS_BLOCK_EXTENSION);
|
||||
|
||||
this.postingsReader = postingsReader;
|
||||
try {
|
||||
this.indexIn = state.directory.openInput(termsIndexFileName, state.context);
|
||||
this.blockIn = state.directory.openInput(termsBlockFileName, state.context);
|
||||
readHeader(indexIn);
|
||||
readHeader(blockIn);
|
||||
this.postingsReader.init(blockIn);
|
||||
seekDir(indexIn);
|
||||
seekDir(blockIn);
|
||||
|
||||
final FieldInfos fieldInfos = state.fieldInfos;
|
||||
final int numFields = blockIn.readVInt();
|
||||
for (int i = 0; i < numFields; i++) {
|
||||
FieldInfo fieldInfo = fieldInfos.fieldInfo(blockIn.readVInt());
|
||||
boolean hasFreq = fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY;
|
||||
long numTerms = blockIn.readVLong();
|
||||
long sumTotalTermFreq = hasFreq ? blockIn.readVLong() : -1;
|
||||
long sumDocFreq = blockIn.readVLong();
|
||||
int docCount = blockIn.readVInt();
|
||||
int longsSize = blockIn.readVInt();
|
||||
FST<Long> index = new FST<Long>(indexIn, PositiveIntOutputs.getSingleton());
|
||||
|
||||
TermsReader current = new TermsReader(fieldInfo, numTerms, sumTotalTermFreq, sumDocFreq, docCount, longsSize, index);
|
||||
TermsReader previous = fields.put(fieldInfo.name, current);
|
||||
checkFieldSummary(state.segmentInfo, current, previous);
|
||||
}
|
||||
} finally {
|
||||
IOUtils.close(indexIn, blockIn);
|
||||
}
|
||||
}
|
||||
|
||||
private int readHeader(IndexInput in) throws IOException {
|
||||
return CodecUtil.checkHeader(in, TempFSTOrdTermsWriter.TERMS_CODEC_NAME,
|
||||
TempFSTOrdTermsWriter.TERMS_VERSION_START,
|
||||
TempFSTOrdTermsWriter.TERMS_VERSION_CURRENT);
|
||||
}
|
||||
private void seekDir(IndexInput in) throws IOException {
|
||||
in.seek(in.length() - 8);
|
||||
in.seek(in.readLong());
|
||||
}
|
||||
private void checkFieldSummary(SegmentInfo info, TermsReader field, TermsReader previous) throws IOException {
|
||||
// #docs with field must be <= #docs
|
||||
if (field.docCount < 0 || field.docCount > info.getDocCount()) {
|
||||
throw new CorruptIndexException("invalid docCount: " + field.docCount + " maxDoc: " + info.getDocCount() + " (resource=" + indexIn + ", " + blockIn + ")");
|
||||
}
|
||||
// #postings must be >= #docs with field
|
||||
if (field.sumDocFreq < field.docCount) {
|
||||
throw new CorruptIndexException("invalid sumDocFreq: " + field.sumDocFreq + " docCount: " + field.docCount + " (resource=" + indexIn + ", " + blockIn + ")");
|
||||
}
|
||||
// #positions must be >= #postings
|
||||
if (field.sumTotalTermFreq != -1 && field.sumTotalTermFreq < field.sumDocFreq) {
|
||||
throw new CorruptIndexException("invalid sumTotalTermFreq: " + field.sumTotalTermFreq + " sumDocFreq: " + field.sumDocFreq + " (resource=" + indexIn + ", " + blockIn + ")");
|
||||
}
|
||||
if (previous != null) {
|
||||
throw new CorruptIndexException("duplicate fields: " + field.fieldInfo.name + " (resource=" + indexIn + ", " + blockIn + ")");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<String> iterator() {
|
||||
return Collections.unmodifiableSet(fields.keySet()).iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
assert field != null;
|
||||
return fields.get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return fields.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
IOUtils.close(postingsReader);
|
||||
} finally {
|
||||
fields.clear();
|
||||
}
|
||||
}
|
||||
|
||||
final class TermsReader extends Terms {
|
||||
final FieldInfo fieldInfo;
|
||||
final long numTerms;
|
||||
final long sumTotalTermFreq;
|
||||
final long sumDocFreq;
|
||||
final int docCount;
|
||||
final int longsSize;
|
||||
final FST<Long> index;
|
||||
|
||||
final int numSkipInfo;
|
||||
final long[] skipInfo;
|
||||
final byte[] statsBlock;
|
||||
final byte[] metaLongsBlock;
|
||||
final byte[] metaBytesBlock;
|
||||
|
||||
TermsReader(FieldInfo fieldInfo, long numTerms, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize, FST<Long> index) throws IOException {
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.numTerms = numTerms;
|
||||
this.sumTotalTermFreq = sumTotalTermFreq;
|
||||
this.sumDocFreq = sumDocFreq;
|
||||
this.docCount = docCount;
|
||||
this.longsSize = longsSize;
|
||||
this.index = index;
|
||||
|
||||
assert (numTerms & (~0xffffffffL)) == 0;
|
||||
final int numBlocks = (int)(numTerms + INTERVAL - 1) / INTERVAL;
|
||||
this.numSkipInfo = longsSize + 3;
|
||||
this.skipInfo = new long[numBlocks * numSkipInfo];
|
||||
this.statsBlock = new byte[(int)blockIn.readVLong()];
|
||||
this.metaLongsBlock = new byte[(int)blockIn.readVLong()];
|
||||
this.metaBytesBlock = new byte[(int)blockIn.readVLong()];
|
||||
|
||||
int last = 0, next = 0;
|
||||
for (int i = 1; i < numBlocks; i++) {
|
||||
next = numSkipInfo * i;
|
||||
for (int j = 0; j < numSkipInfo; j++) {
|
||||
skipInfo[next + j] = skipInfo[last + j] + blockIn.readVLong();
|
||||
}
|
||||
last = next;
|
||||
}
|
||||
blockIn.readBytes(statsBlock, 0, statsBlock.length);
|
||||
blockIn.readBytes(metaLongsBlock, 0, metaLongsBlock.length);
|
||||
blockIn.readBytes(metaBytesBlock, 0, metaBytesBlock.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
public boolean hasFreqs() {
|
||||
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasOffsets() {
|
||||
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPositions() {
|
||||
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPayloads() {
|
||||
return fieldInfo.hasPayloads();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long size() {
|
||||
return numTerms;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumTotalTermFreq() {
|
||||
return sumTotalTermFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumDocFreq() throws IOException {
|
||||
return sumDocFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getDocCount() throws IOException {
|
||||
return docCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
return new SegmentTermsEnum();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
|
||||
return super.intersect(compiled, startTerm);
|
||||
}
|
||||
|
||||
// Only wraps common operations for PBF interact
|
||||
abstract class BaseTermsEnum extends TermsEnum {
|
||||
/* Current term, null when enum ends or unpositioned */
|
||||
BytesRef term;
|
||||
|
||||
/* Current term's ord, starts from 0 */
|
||||
long ord;
|
||||
|
||||
/* Current term stats + decoded metadata (customized by PBF) */
|
||||
final TempTermState state;
|
||||
|
||||
/* Datainput to load stats & metadata */
|
||||
final ByteArrayDataInput statsReader = new ByteArrayDataInput();
|
||||
final ByteArrayDataInput metaLongsReader = new ByteArrayDataInput();
|
||||
final ByteArrayDataInput metaBytesReader = new ByteArrayDataInput();
|
||||
|
||||
/** Decodes metadata into customized term state */
|
||||
abstract void decodeMetaData() throws IOException;
|
||||
|
||||
BaseTermsEnum() throws IOException {
|
||||
this.state = postingsReader.newTermState();
|
||||
this.term = null;
|
||||
this.statsReader.reset(statsBlock);
|
||||
this.metaLongsReader.reset(metaLongsBlock);
|
||||
this.metaBytesReader.reset(metaBytesBlock);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermState termState() throws IOException {
|
||||
decodeMetaData();
|
||||
return state.clone();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef term() {
|
||||
return term;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq() throws IOException {
|
||||
return state.docFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long totalTermFreq() throws IOException {
|
||||
return state.totalTermFreq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
|
||||
decodeMetaData();
|
||||
return postingsReader.docs(fieldInfo, state, liveDocs, reuse, flags);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
|
||||
if (!hasPositions()) {
|
||||
return null;
|
||||
}
|
||||
decodeMetaData();
|
||||
return postingsReader.docsAndPositions(fieldInfo, state, liveDocs, reuse, flags);
|
||||
}
|
||||
|
||||
// nocommit: do we need this? for SegmentTermsEnum, we can maintain
|
||||
// a stack to record how current term is constructed on FST, (and ord on each alphabet)
|
||||
// so that during seek we don't have to start from the first arc.
|
||||
// however, we'll be implementing a new fstEnum instead of wrapping current one.
|
||||
//
|
||||
// nocommit: this can also be achieved by making use of Util.getByOutput()
|
||||
@Override
|
||||
public void seekExact(long ord) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ord() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// Iterates through all terms in this field
|
||||
private final class SegmentTermsEnum extends BaseTermsEnum {
|
||||
final BytesRefFSTEnum<Long> fstEnum;
|
||||
|
||||
/* True when current term's metadata is decoded */
|
||||
boolean decoded;
|
||||
|
||||
/* True when current enum is 'positioned' by seekExact(TermState) */
|
||||
boolean seekPending;
|
||||
|
||||
/* To which block is buffered */
|
||||
int statsBlockOrd;
|
||||
int metaBlockOrd;
|
||||
|
||||
/* Current buffered metadata (long[] & byte[]) */
|
||||
long[][] longs;
|
||||
int[] bytesStart;
|
||||
int[] bytesLength;
|
||||
|
||||
/* Current buffered stats (df & ttf) */
|
||||
int[] docFreq;
|
||||
long[] totalTermFreq;
|
||||
|
||||
SegmentTermsEnum() throws IOException {
|
||||
super();
|
||||
this.fstEnum = new BytesRefFSTEnum<Long>(index);
|
||||
this.decoded = false;
|
||||
this.seekPending = false;
|
||||
this.longs = new long[INTERVAL][longsSize];
|
||||
this.bytesStart = new int[INTERVAL];
|
||||
this.bytesLength = new int[INTERVAL];
|
||||
this.docFreq = new int[INTERVAL];
|
||||
this.totalTermFreq = new long[INTERVAL];
|
||||
this.statsBlockOrd = -1;
|
||||
this.metaBlockOrd = -1;
|
||||
}
|
||||
|
||||
void refillStats() throws IOException {
|
||||
final int offset = statsBlockOrd * numSkipInfo;
|
||||
final int statsFP = (int)skipInfo[offset];
|
||||
statsReader.setPosition(statsFP);
|
||||
if (!hasFreqs()) {
|
||||
Arrays.fill(totalTermFreq, -1);
|
||||
}
|
||||
for (int i = 0; i < INTERVAL && !statsReader.eof(); i++) {
|
||||
int code = statsReader.readVInt();
|
||||
if (hasFreqs()) {
|
||||
docFreq[i] = (code >>> 1);
|
||||
if ((code & 1) == 1) {
|
||||
totalTermFreq[i] = docFreq[i];
|
||||
} else {
|
||||
totalTermFreq[i] = docFreq[i] + statsReader.readVLong();
|
||||
}
|
||||
} else {
|
||||
docFreq[i] = code;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void refillMetadata() throws IOException {
|
||||
final int offset = metaBlockOrd * numSkipInfo;
|
||||
final int metaLongsFP = (int)skipInfo[offset + 1];
|
||||
final int metaBytesFP = (int)skipInfo[offset + 2];
|
||||
metaLongsReader.setPosition(metaLongsFP);
|
||||
bytesStart[0] = metaBytesFP;
|
||||
for (int j = 0; j < longsSize; j++) {
|
||||
longs[0][j] = skipInfo[offset + 3 + j] + metaLongsReader.readVLong();
|
||||
}
|
||||
bytesLength[0] = (int)metaLongsReader.readVLong();
|
||||
for (int i = 1; i < INTERVAL && !metaLongsReader.eof(); i++) {
|
||||
bytesStart[i] = bytesStart[i-1] + bytesLength[i-1];
|
||||
for (int j = 0; j < longsSize; j++) {
|
||||
longs[i][j] = longs[i-1][j] + metaLongsReader.readVLong();
|
||||
}
|
||||
bytesLength[i] = (int)metaLongsReader.readVLong();
|
||||
}
|
||||
}
|
||||
|
||||
void decodeStats() throws IOException {
|
||||
final int upto = (int)ord % INTERVAL;
|
||||
final int oldBlockOrd = statsBlockOrd;
|
||||
statsBlockOrd = (int)ord / INTERVAL;
|
||||
if (oldBlockOrd != statsBlockOrd) {
|
||||
refillStats();
|
||||
}
|
||||
state.docFreq = docFreq[upto];
|
||||
state.totalTermFreq = totalTermFreq[upto];
|
||||
}
|
||||
|
||||
// Let PBF decode metadata from long[] and byte[]
|
||||
@Override
|
||||
void decodeMetaData() throws IOException {
|
||||
final int upto = (int)ord % INTERVAL;
|
||||
final int oldBlockOrd = metaBlockOrd;
|
||||
metaBlockOrd = (int)ord / INTERVAL;
|
||||
if (metaBlockOrd != oldBlockOrd) {
|
||||
refillMetadata();
|
||||
}
|
||||
if (!decoded && !seekPending) {
|
||||
metaBytesReader.reset(metaBytesBlock, bytesStart[upto], bytesLength[upto]);
|
||||
postingsReader.decodeTerm(longs[upto], metaBytesReader, fieldInfo, state);
|
||||
decoded = true;
|
||||
}
|
||||
}
|
||||
|
||||
// Update current enum according to FSTEnum
|
||||
void updateEnum(final InputOutput<Long> pair) throws IOException {
|
||||
if (pair == null) {
|
||||
term = null;
|
||||
} else {
|
||||
term = pair.input;
|
||||
ord = pair.output;
|
||||
decodeStats();
|
||||
}
|
||||
decoded = false;
|
||||
seekPending = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef next() throws IOException {
|
||||
if (seekPending) { // previously positioned, but termOutputs not fetched
|
||||
seekPending = false;
|
||||
SeekStatus status = seekCeil(term, false);
|
||||
assert status == SeekStatus.FOUND; // must positioned on valid term
|
||||
}
|
||||
updateEnum(fstEnum.next());
|
||||
return term;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean seekExact(BytesRef target, boolean useCache) throws IOException {
|
||||
updateEnum(fstEnum.seekExact(target));
|
||||
return term != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekStatus seekCeil(BytesRef target, boolean useCache) throws IOException {
|
||||
updateEnum(fstEnum.seekCeil(target));
|
||||
if (term == null) {
|
||||
return SeekStatus.END;
|
||||
} else {
|
||||
return term.equals(target) ? SeekStatus.FOUND : SeekStatus.NOT_FOUND;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seekExact(BytesRef target, TermState otherState) {
|
||||
if (!target.equals(term)) {
|
||||
state.copyFrom(otherState);
|
||||
term = BytesRef.deepCopyOf(target);
|
||||
seekPending = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static<T> void walk(FST<T> fst) throws IOException {
|
||||
final ArrayList<FST.Arc<T>> queue = new ArrayList<FST.Arc<T>>();
|
||||
final BitSet seen = new BitSet();
|
||||
final FST.BytesReader reader = fst.getBytesReader();
|
||||
final FST.Arc<T> startArc = fst.getFirstArc(new FST.Arc<T>());
|
||||
queue.add(startArc);
|
||||
while (!queue.isEmpty()) {
|
||||
final FST.Arc<T> arc = queue.remove(0);
|
||||
final long node = arc.target;
|
||||
if (FST.targetHasArcs(arc) && !seen.get((int) node)) {
|
||||
seen.set((int) node);
|
||||
fst.readFirstRealTargetArc(node, arc, reader);
|
||||
while (true) {
|
||||
queue.add(new FST.Arc<T>().copyFrom(arc));
|
||||
if (arc.isLast()) {
|
||||
break;
|
||||
} else {
|
||||
fst.readNextRealArc(arc, reader);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,267 @@
|
|||
package org.apache.lucene.codecs.temp;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
import org.apache.lucene.codecs.TempPostingsWriterBase;
|
||||
import org.apache.lucene.codecs.PostingsConsumer;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.TermsConsumer;
|
||||
import org.apache.lucene.codecs.TermStats;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
|
||||
/** FST based term dict, only ords is hold in FST,
|
||||
* other metadata encoded into single byte block */
|
||||
|
||||
public class TempFSTOrdTermsWriter extends FieldsConsumer {
|
||||
static final String TERMS_INDEX_EXTENSION = "tix";
|
||||
static final String TERMS_BLOCK_EXTENSION = "tbk";
|
||||
static final String TERMS_CODEC_NAME = "FST_ORD_TERMS_DICT";
|
||||
public static final int TERMS_VERSION_START = 0;
|
||||
public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_START;
|
||||
public static final int SKIP_INTERVAL = 8;
|
||||
static final boolean TEST = false;
|
||||
|
||||
final TempPostingsWriterBase postingsWriter;
|
||||
final FieldInfos fieldInfos;
|
||||
final List<FieldMetaData> fields = new ArrayList<FieldMetaData>();
|
||||
IndexOutput blockOut = null;
|
||||
IndexOutput indexOut = null; // nocommit: hmm, do we really need two streams?
|
||||
|
||||
public TempFSTOrdTermsWriter(SegmentWriteState state, TempPostingsWriterBase postingsWriter) throws IOException {
|
||||
final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
|
||||
final String termsBlockFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_BLOCK_EXTENSION);
|
||||
|
||||
this.postingsWriter = postingsWriter;
|
||||
this.fieldInfos = state.fieldInfos;
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
this.indexOut = state.directory.createOutput(termsIndexFileName, state.context);
|
||||
this.blockOut = state.directory.createOutput(termsBlockFileName, state.context);
|
||||
writeHeader(indexOut);
|
||||
writeHeader(blockOut);
|
||||
this.postingsWriter.start(blockOut);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(indexOut, blockOut);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsConsumer addField(FieldInfo field) throws IOException {
|
||||
return new TermsWriter(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
IOException ioe = null;
|
||||
try {
|
||||
final long indexDirStart = indexOut.getFilePointer();
|
||||
final long blockDirStart = blockOut.getFilePointer();
|
||||
|
||||
// write field summary
|
||||
blockOut.writeVInt(fields.size());
|
||||
for (FieldMetaData field : fields) {
|
||||
blockOut.writeVInt(field.fieldInfo.number);
|
||||
blockOut.writeVLong(field.numTerms);
|
||||
if (field.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
|
||||
blockOut.writeVLong(field.sumTotalTermFreq);
|
||||
}
|
||||
blockOut.writeVLong(field.sumDocFreq);
|
||||
blockOut.writeVInt(field.docCount);
|
||||
blockOut.writeVInt(field.longsSize);
|
||||
blockOut.writeVLong(field.statsOut.getFilePointer());
|
||||
blockOut.writeVLong(field.metaLongsOut.getFilePointer());
|
||||
blockOut.writeVLong(field.metaBytesOut.getFilePointer());
|
||||
|
||||
field.skipOut.writeTo(blockOut);
|
||||
field.statsOut.writeTo(blockOut);
|
||||
field.metaLongsOut.writeTo(blockOut);
|
||||
field.metaBytesOut.writeTo(blockOut);
|
||||
field.dict.save(indexOut);
|
||||
}
|
||||
writeTrailer(indexOut, indexDirStart);
|
||||
writeTrailer(blockOut, blockDirStart);
|
||||
} catch (IOException ioe2) {
|
||||
ioe = ioe2;
|
||||
} finally {
|
||||
IOUtils.closeWhileHandlingException(ioe, blockOut, indexOut, postingsWriter);
|
||||
}
|
||||
}
|
||||
|
||||
private void writeHeader(IndexOutput out) throws IOException {
|
||||
CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);
|
||||
}
|
||||
private void writeTrailer(IndexOutput out, long dirStart) throws IOException {
|
||||
out.writeLong(dirStart);
|
||||
}
|
||||
|
||||
// nocommit: nuke this? we don't need to buffer so much data,
|
||||
// since close() can do this naturally
|
||||
private static class FieldMetaData {
|
||||
public FieldInfo fieldInfo;
|
||||
public long numTerms;
|
||||
public long sumTotalTermFreq;
|
||||
public long sumDocFreq;
|
||||
public int docCount;
|
||||
public int longsSize;
|
||||
public FST<Long> dict;
|
||||
|
||||
// nocommit: block encode each part
|
||||
// (so that we'll have metaLongsOut[])
|
||||
public RAMOutputStream skipOut; // vint encode next skip point (all values start from 0, fully decoded when reading)
|
||||
public RAMOutputStream statsOut; // vint encode df, (ttf-df)
|
||||
public RAMOutputStream metaLongsOut; // vint encode metalongs[0~(fieldSize-1)] and metaBytes[startFPdelta, len]
|
||||
public RAMOutputStream metaBytesOut; // put all bytes blob here
|
||||
}
|
||||
|
||||
final class TermsWriter extends TermsConsumer {
|
||||
private final Builder<Long> builder;
|
||||
private final PositiveIntOutputs outputs;
|
||||
private final FieldInfo fieldInfo;
|
||||
private final int longsSize;
|
||||
private long numTerms;
|
||||
|
||||
private final IntsRef scratchTerm = new IntsRef();
|
||||
private final RAMOutputStream statsOut = new RAMOutputStream();
|
||||
private final RAMOutputStream metaLongsOut = new RAMOutputStream();
|
||||
private final RAMOutputStream metaBytesOut = new RAMOutputStream();
|
||||
|
||||
private final RAMOutputStream skipOut = new RAMOutputStream();
|
||||
private long lastBlockStatsFP;
|
||||
private long lastBlockMetaLongsFP;
|
||||
private long lastBlockMetaBytesFP;
|
||||
private long[] lastBlockLongs;
|
||||
|
||||
private long[] lastLongs;
|
||||
private long lastMetaBytesFP;
|
||||
|
||||
TermsWriter(FieldInfo fieldInfo) {
|
||||
this.numTerms = 0;
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.longsSize = postingsWriter.setField(fieldInfo);
|
||||
this.outputs = PositiveIntOutputs.getSingleton();
|
||||
this.builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
|
||||
|
||||
this.lastBlockStatsFP = 0;
|
||||
this.lastBlockMetaLongsFP = 0;
|
||||
this.lastBlockMetaBytesFP = 0;
|
||||
this.lastBlockLongs = new long[longsSize];
|
||||
|
||||
this.lastLongs = new long[longsSize];
|
||||
this.lastMetaBytesFP = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsConsumer startTerm(BytesRef text) throws IOException {
|
||||
postingsWriter.startTerm();
|
||||
return postingsWriter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finishTerm(BytesRef text, TermStats stats) throws IOException {
|
||||
if (numTerms > 0 && numTerms % SKIP_INTERVAL == 0) {
|
||||
bufferSkip();
|
||||
}
|
||||
// write term meta data into fst
|
||||
final long longs[] = new long[longsSize];
|
||||
final long delta = stats.totalTermFreq - stats.docFreq;
|
||||
if (stats.totalTermFreq > 0) {
|
||||
if (delta == 0) {
|
||||
statsOut.writeVInt(stats.docFreq<<1|1);
|
||||
} else {
|
||||
statsOut.writeVInt(stats.docFreq<<1|0);
|
||||
statsOut.writeVLong(stats.totalTermFreq-stats.docFreq);
|
||||
}
|
||||
} else {
|
||||
statsOut.writeVInt(stats.docFreq);
|
||||
}
|
||||
postingsWriter.finishTerm(longs, metaBytesOut, stats);
|
||||
for (int i = 0; i < longsSize; i++) {
|
||||
metaLongsOut.writeVLong(longs[i] - lastLongs[i]);
|
||||
}
|
||||
metaLongsOut.writeVLong(metaBytesOut.getFilePointer() - lastMetaBytesFP);
|
||||
|
||||
builder.add(Util.toIntsRef(text, scratchTerm), numTerms);
|
||||
numTerms++;
|
||||
|
||||
lastMetaBytesFP = metaBytesOut.getFilePointer();
|
||||
lastLongs = longs;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
|
||||
if (numTerms > 0) {
|
||||
final FieldMetaData metadata = new FieldMetaData();
|
||||
metadata.fieldInfo = fieldInfo;
|
||||
metadata.numTerms = numTerms;
|
||||
metadata.sumTotalTermFreq = sumTotalTermFreq;
|
||||
metadata.sumDocFreq = sumDocFreq;
|
||||
metadata.docCount = docCount;
|
||||
metadata.longsSize = longsSize;
|
||||
metadata.skipOut = skipOut;
|
||||
metadata.statsOut = statsOut;
|
||||
metadata.metaLongsOut = metaLongsOut;
|
||||
metadata.metaBytesOut = metaBytesOut;
|
||||
metadata.dict = builder.finish();
|
||||
fields.add(metadata);
|
||||
}
|
||||
}
|
||||
|
||||
private void bufferSkip() throws IOException {
|
||||
skipOut.writeVLong(statsOut.getFilePointer() - lastBlockStatsFP);
|
||||
skipOut.writeVLong(metaLongsOut.getFilePointer() - lastBlockMetaLongsFP);
|
||||
skipOut.writeVLong(metaBytesOut.getFilePointer() - lastBlockMetaBytesFP);
|
||||
for (int i = 0; i < longsSize; i++) {
|
||||
skipOut.writeVLong(lastLongs[i] - lastBlockLongs[i]);
|
||||
}
|
||||
lastBlockStatsFP = statsOut.getFilePointer();
|
||||
lastBlockMetaLongsFP = metaLongsOut.getFilePointer();
|
||||
lastBlockMetaBytesFP = metaBytesOut.getFilePointer();
|
||||
lastBlockLongs = lastLongs;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,3 +17,4 @@ org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat
|
|||
org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat
|
||||
org.apache.lucene.codecs.temp.TempBlockPostingsFormat
|
||||
org.apache.lucene.codecs.temp.TempFSTPostingsFormat
|
||||
org.apache.lucene.codecs.temp.TempFSTOrdPostingsFormat
|
||||
|
|
|
@ -1015,7 +1015,8 @@ public class TestIndexWriterReader extends LuceneTestCase {
|
|||
// Don't proceed if picked Codec is in the list of illegal ones.
|
||||
final String format = _TestUtil.getPostingsFormat("f");
|
||||
assumeFalse("Format: " + format + " does not support ReaderTermsIndexDivisor!",
|
||||
(format.equals("SimpleText") || format.equals("Memory") || format.equals("Direct") || format.equals("TempFST")));
|
||||
(format.equals("SimpleText") || format.equals("Memory") || format.equals("Direct") ||
|
||||
format.equals("TempFST") || format.equals("TempFSTOrd")));
|
||||
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(dir, conf);
|
||||
|
|
Loading…
Reference in New Issue